CEP-28: Apache Cassandra Analytics

This is the initial commit for the Apache Cassandra Analytics project
where we support reading and writing bulk data from Apache Cassandra from
Spark.

Patch by James Berragan, Doug Rohrer; Reviewed by Dinesh Joshi, Yifan Cai for CASSANDRA-16222

Co-authored-by: James Berragan <jberragan@apple.com>
Co-authored-by: Doug Rohrer <drohrer@apple.com>
Co-authored-by: Saranya Krishnakumar <saranya_k@apple.com>
Co-authored-by: Francisco Guerrero <francisco.guerrero@apple.com>
Co-authored-by: Yifan Cai <ycai@apache.org>
Co-authored-by: Jyothsna Konisa <jkonisa@apple.com>
Co-authored-by: Yuriy Semchyshyn <ysemchyshyn@apple.com>
Co-authored-by: Dinesh Joshi <djoshi@apache.org>
diff --git a/.gitignore b/.gitignore
new file mode 100644
index 0000000..c8e73d0
--- /dev/null
+++ b/.gitignore
@@ -0,0 +1,101 @@
+# 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.
+
+# Compiled class files
+*.class
+
+# Log files
+*.log
+
+# BlueJ files
+*.ctxt
+
+# Mobile Tools for Java (J2ME)
+.mtj.tmp/
+
+# Package files
+*.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*
+
+# Build output directories
+.dist
+/target
+*/target
+/build
+*/build
+
+# IntelliJ files and directories
+out
+.idea
+*.ipr
+*.iws
+*.iml
+atlassian-ide-plugin.xml
+
+# Gradle
+.gradle/
+.out/
+
+# Docker tests
+nohup.out
+dockertests/libs
+test_results/
+
+# generted certs
+sbr_keystore
+sbr_keystore_pass
+sbr_truststore
+sbr_truststore_pass
+truststore.jks
+
+*.hprof
+
+__pycache__/
+
+.vscode/
+
+cdc/bin
+
+# Miscellaneous
+/env/**
+build.log
+.gradle
+.packager
+.shelf
+logs
+generated
+**/build/**
+metastore_db/
+*.pyc
+.cache/
+ttr.log
+buildcache/
+.testdata*/
+dev-lo.properties
+pytests/.env/**/*
+pytests/.pytest_cache
+cassandra-analytics-build.properties
+
+# MacOS
+.DS_Store
diff --git a/DEV-README.md b/DEV-README.md
new file mode 100644
index 0000000..d85755c
--- /dev/null
+++ b/DEV-README.md
@@ -0,0 +1,71 @@
+# Cassandra Analytics
+
+Cassandra Analytics supports Spark 2 (Scala 2.11 and 2.12) and Spark 3 (Scala 2.12).
+
+This project uses Gradle as the dependency management and build framework.
+
+## Building
+
+Cassandra Analytics will build for Spark 2 and Scala 2.11 by default.
+
+Navigate to the top-level directory for this project:
+
+```shell
+./gradlew clean package
+```
+
+### Spark 2 and Scala 2.12
+
+To build for Scala 2.12, set the profile by exporting `SCALA_VERSION=2.12`:
+
+```shell
+export SCALA_VERSION=2.12
+./gradlew clean package
+```
+
+### Spark 3 and Scala 2.12
+
+To build for Spark 3 and Scala 2.12, export both `SCALA_VERSION=2.12` and `SPARK_VERSION=3`:
+
+```shell
+export SCALA_VERSION=2.12
+export SPARK_VERSION=3
+./gradlew clean package
+```
+
+### Git hooks (optional)
+
+To enable git hooks, run the following command at project root. 
+
+```shell
+git config core.hooksPath githooks
+```
+
+## IntelliJ
+
+The project is well-supported in IntelliJ.
+
+Run the following profile to copy code style used for this project:
+
+```shell
+./gradlew copyCodeStyle
+```
+
+The project has different sources for Spark 2 and Spark 3.
+
+Spark 2 uses the `org.apache.spark.sql.sources.v2` APIs that have been deprecated in Spark 3.
+
+Spark 3 uses new APIs that live in the `org.apache.spark.sql.connector.read` namespace.
+
+By default, the project will load Spark 2 sources, but you can switch between sources by modifying the `gradle.properties` file.
+
+For Spark 3, use the following in `gradle.properties`:
+
+```properties
+scala=2.12
+spark=3
+```
+
+And then load Gradle changes (on Mac, the shortcut to load Gradle changes is <kbd>Command</kbd> + <kbd>Shift</kbd> + <kbd>I</kbd>).
+
+This will make the IDE pick up the Spark 3 sources, and you should now be able to develop against Spark 3 as well.
diff --git a/LICENSE.txt b/LICENSE.txt
new file mode 100644
index 0000000..d645695
--- /dev/null
+++ b/LICENSE.txt
@@ -0,0 +1,202 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   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.
diff --git a/README.md b/README.md
new file mode 100644
index 0000000..3820361
--- /dev/null
+++ b/README.md
@@ -0,0 +1,37 @@
+# Cassandra Analytics 
+
+## Cassandra Spark Bulk Reader
+
+The open-source repository for the Cassandra Spark Bulk Reader. This library allows integration between Cassandra and Spark job, allowing users to run arbitrary Spark jobs against a Cassandra cluster securely and consistently. 
+
+This project contains the necessary open-source implementations to connect to a Cassandra cluster and read the data into Spark.
+
+For example usage, see the example repository; sample steps:
+
+```scala
+import org.apache.cassandra.spark.sparksql.CassandraDataSource
+import org.apache.spark.sql.SparkSession
+
+val sparkSession = SparkSession.builder.getOrCreate()
+val df = sparkSession.read.format("org.apache.cassandra.spark.sparksql.CassandraDataSource")
+                          .option("sidecar_instances", "localhost,localhost2,localhost3")
+                          .option("keyspace", "sbr_tests")
+                          .option("table", "basic_test")
+                          .option("DC", "datacenter1")
+                          .option("createSnapshot", true)
+                          .option("numCores", 4)
+                          .load()
+```
+   
+## Cassandra Spark Bulk Writer
+
+The Cassandra Spark Bulk Writer allows for high-speed data ingest to Cassandra clusters running Cassandra 3.0 and 4.0.
+
+If you are a consumer of the Cassandra Spark Bulk Writer, please see our end-user documentation: usage instructions, FAQs, troubleshooting guides, and release notes.
+
+Developers interested in contributing to the SBW, please see the [DEV-README](DEV-README.md).
+
+## Getting Started
+
+For example usage, see the [example repository](cassandra-analytics-core-example/README.md). This example covers both
+setting up Cassandra 4.0, Apache Sidecar, and running a Spark Bulk Reader and Spark Bulk Writer job.
diff --git a/build.gradle b/build.gradle
new file mode 100644
index 0000000..ff72ca1
--- /dev/null
+++ b/build.gradle
@@ -0,0 +1,144 @@
+/*
+ * 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.
+ */
+
+buildscript {
+  dependencies {
+    classpath(group: 'com.github.jengelman.gradle.plugins', name: 'shadow', version: '6.1.0')
+  }
+}
+
+plugins {
+  id 'java'
+  id 'java-library'
+  id 'com.github.johnrengelman.shadow' version '5.1.0'
+}
+
+ext.scalaLabel = System.getenv("SCALA_VERSION") ?: "${scala}"
+println("Scala version: ${ext.scalaLabel}")
+
+ext.sparkLabel = (System.getenv("SPARK_VERSION") ?: "${spark}").split(/\./)[0]
+println("Spark version: ${ext.sparkLabel}")
+
+ext.jdkLabel = System.getenv("JDK_VERSION") ?: "${analyticsJDKLevel}"
+println("Java source/target compatibility level: ${ext.jdkLabel}")
+
+def profile = "profiles/scala-${ext.scalaLabel}-spark-${ext.sparkLabel}-jdk-${ext.jdkLabel}.gradle"
+if (!file(profile).exists()) {
+  throw new InvalidUserDataException("Profile ${profile} does not exist, which indicates this combination of Scala, Spark, and Java is unsupported.\n" +
+          "Please either add an appropriate profile if this combination should work, or use a different one.")
+}
+
+println("Using profile ${profile}")
+apply(from: profile)
+
+apply(plugin: 'idea')
+
+task copyCodeStyle(type: Copy) {
+  from('ide/idea/codeStyleSettings.xml')
+  into('.idea')
+}
+
+tasks.idea.dependsOn(tasks.copyCodeStyle)
+
+task copyInspections(type: Copy) {
+  from('ide/idea/Project_Default.xml')
+  into('.idea/inspectionProfiles')
+}
+tasks.idea.dependsOn(tasks.copyInspections)
+
+subprojects {
+  apply(plugin: 'java-library')
+  apply(plugin: 'checkstyle')
+
+  sourceCompatibility = "${project.rootProject.ext.jdkLabel}"
+  targetCompatibility = "${project.rootProject.ext.jdkLabel}"
+
+  tasks.withType(JavaCompile) {
+    if ("${project.rootProject.ext.jdkLabel}" == '1.8') {
+      // Unfortunately, we can't use release here. We can only use public APIs when using release.
+      // org.apache.cassandra.spark.bulkwriter.util.FastByteOperations uses sun.misc.Unsafe which is causing the build
+      // to fail.
+      options.compilerArgs += ['-source', '8', '-target', '8']
+    } else {
+      options.release = 11
+    }
+  }
+
+  archivesBaseName = "${project.name}"
+  if ("${project.rootProject.ext.sparkLabel}" == '3') {
+    archivesBaseName = "${archivesBaseName}_spark3"
+  }
+  archivesBaseName = "${archivesBaseName}_${scalaMajorVersion}"
+
+  if ("${project.rootProject.ext.jdkLabel}" == '1.8') {
+    if ("${version}".contains('-SNAPSHOT')) {
+      version = "${version}".replace('-SNAPSHOT', '-jdk8-SNAPSHOT')
+    } else {
+      version = "${version}-jdk8"
+    }
+  }
+
+  repositories {
+    // uncomment the line below for local development
+    mavenLocal()
+  }
+
+  dependencies {
+    compileOnly(group: 'com.intellij', name: 'annotations', version: "${project.rootProject.intellijVersion}")
+    testCompileOnly(group: 'com.intellij', name: 'annotations', version: "${project.rootProject.intellijVersion}")
+  }
+
+  sourceSets {
+    main {
+      java {
+        srcDirs += ["src/main/spark${project.rootProject.ext.sparkLabel}",
+                    "src/main/scala-${scalaMajorVersion}-spark-${sparkMajorVersion}"]
+      }
+    }
+    test {
+      java {
+        srcDirs += ["src/test/spark${project.rootProject.ext.sparkLabel}",
+                    "src/test/scala-${scalaMajorVersion}-spark-${sparkMajorVersion}"]
+      }
+    }
+  }
+
+  test {
+    if (JavaVersion.current().isJava11Compatible()) {
+      def JDK11_OPTIONS = ['-Djdk.attach.allowAttachSelf=true',
+                           '--add-exports', 'java.base/jdk.internal.misc=ALL-UNNAMED',
+                           '--add-exports', 'java.base/jdk.internal.ref=ALL-UNNAMED',
+                           '--add-exports', 'java.base/sun.nio.ch=ALL-UNNAMED',
+                           '--add-exports', 'java.management.rmi/com.sun.jmx.remote.internal.rmi=ALL-UNNAMED',
+                           '--add-exports', 'java.rmi/sun.rmi.registry=ALL-UNNAMED',
+                           '--add-exports', 'java.rmi/sun.rmi.server=ALL-UNNAMED',
+                           '--add-exports', 'java.sql/java.sql=ALL-UNNAMED',
+                           '--add-opens', 'java.base/java.lang.module=ALL-UNNAMED',
+                           '--add-opens', 'java.base/jdk.internal.loader=ALL-UNNAMED',
+                           '--add-opens', 'java.base/jdk.internal.ref=ALL-UNNAMED',
+                           '--add-opens', 'java.base/jdk.internal.reflect=ALL-UNNAMED',
+                           '--add-opens', 'java.base/jdk.internal.math=ALL-UNNAMED',
+                           '--add-opens', 'java.base/jdk.internal.module=ALL-UNNAMED',
+                           '--add-opens', 'java.base/jdk.internal.util.jar=ALL-UNNAMED',
+                           '--add-opens', 'jdk.management/com.sun.management.internal=ALL-UNNAMED']
+      jvmArgs(JDK11_OPTIONS)
+      println("JVM arguments for $project.name are $allJvmArgs")
+    }
+  }
+}
diff --git a/cassandra-analytics-core-example/README.md b/cassandra-analytics-core-example/README.md
new file mode 100644
index 0000000..4e219a8
--- /dev/null
+++ b/cassandra-analytics-core-example/README.md
@@ -0,0 +1,231 @@
+# Sample Cassandra Data with Spark Bulk Analytics Job
+
+This sub-project showcases the Cassandra Spark Bulk Analytics read and write functionality. The job writes data to a
+Cassandra table using the bulk writer functionality, and then it reads the data using the bulk reader functionality.
+
+## Requirements
+
+- Java 11
+- A running Cassandra 4.0 cluster
+- A running Cassandra Sidecar service
+
+## Setup
+
+This example requires a Cassandra 4.0 cluster running, we recommend a 3-node cluster. Additionally, it requires the
+latest version of Cassandra Sidecar that we will run from source.
+
+### Step 1: Cassandra using CCM
+
+The easiest way to provision a Cassandra cluster is by using Cassandra Cluster Manager or
+[CCM](https://github.com/riptano/ccm). Follow the configuration instructions for CCM and provision a new test cluster.
+
+For example,
+
+```shell
+ccm create test --version=4.1.1 --nodes=3
+```
+
+> **Note**
+> If you are using macOS, explicit configurations of interface aliases are needed as follows:
+
+```shell
+sudo ifconfig lo0 alias 127.0.0.2
+sudo ifconfig lo0 alias 127.0.0.3
+sudo bash -c 'echo "127.0.0.2  localhost2" >> /etc/hosts'
+sudo bash -c 'echo "127.0.0.3  localhost3" >> /etc/hosts'
+```
+
+After configuring the network interfaces run:
+
+```shell
+ccm start
+```
+
+Verify that your Cassandra cluster is configured and running successfully.
+
+### Step 2: Configure and Run Cassandra Sidecar
+
+In this step, we will clone and configure the Cassandra Sidecar project. Finally, we will run Sidecar which will be
+connecting to our local Cassandra 3-node cluster.
+
+```shell
+git clone https://github.com/frankgh/cassandra-sidecar/tree/CEP-28-bulk-apis
+cd cassandra-sidecar
+```
+
+Configure the `main/dist/sidecar.yaml` file for your local environment. You will most likely only need to configure
+the `cassandra_instances` section in your file pointing to your local Cassandra data directories. Here is what my
+configuration looks like for this tutorial:
+
+```yaml
+cassandra_instances:
+  - id: 1
+    host: localhost
+    port: 9042
+    data_dirs: <your_ccm_parent_path>/.ccm/test/node1/data0
+    uploads_staging_dir: <your_ccm_parent_path>/.ccm/test/node1/sstable-staging
+    jmx_host: 127.0.0.1
+    jmx_port: 7100
+    jmx_ssl_enabled: false
+  - id: 2
+    host: localhost2
+    port: 9042
+    data_dirs: <your_ccm_parent_path>/.ccm/test/node2/data0
+    uploads_staging_dir: <your_ccm_parent_path>/.ccm/test/node2/sstable-staging
+    jmx_host: 127.0.0.1
+    jmx_port: 7200
+    jmx_ssl_enabled: false
+  - id: 3
+    host: localhost3
+    port: 9042
+    data_dirs: <your_ccm_parent_path>/.ccm/test/node3/data0
+    uploads_staging_dir: <your_ccm_parent_path>/.ccm/test/node3/sstable-staging
+    jmx_host: 127.0.0.1
+    jmx_port: 7300
+    jmx_ssl_enabled: false
+```
+
+I have a 3 node setup, so I configure Sidecar for those 3 nodes. CCM creates the Cassandra cluster under
+`${HOME}/.ccm/test`, so I update my `data_dirs` and `uploads_staging_dir` configuration to use my local path.
+
+Next, create the `uploads_staging_dir` where Sidecar will stage SSTables coming from Cassandra Spark bulk writer.
+In my case, I have decided to keep the `sstable-staging` directory inside each of the node's directories.
+
+```shell
+mkdir -p ${HOME}/.ccm/test/node1/sstable-staging
+mkdir -p ${HOME}/.ccm/test/node2/sstable-staging
+mkdir -p ${HOME}/.ccm/test/node3/sstable-staging
+```
+
+Finally, run Cassandra Sidecar, we skip running integration tests because we need docker for integration tests. You
+can opt to run integration tests if you have docker running in your local environment.
+
+```shell
+user:~$ ./gradlew run -x integrationTest
+> Task :common:compileJava UP-TO-DATE
+> Task :cassandra40:compileJava UP-TO-DATE
+> Task :compileJava UP-TO-DATE
+> Task :processResources UP-TO-DATE
+> Task :classes UP-TO-DATE
+> Task :jar UP-TO-DATE
+> Task :startScripts UP-TO-DATE
+> Task :cassandra40:processResources NO-SOURCE
+> Task :cassandra40:classes UP-TO-DATE
+> Task :cassandra40:jar UP-TO-DATE
+> Task :common:processResources NO-SOURCE
+> Task :common:classes UP-TO-DATE
+> Task :common:jar UP-TO-DATE
+> Task :distTar
+> Task :distZip
+> Task :assemble
+> Task :common:compileTestFixturesJava UP-TO-DATE
+> Task :compileTestFixturesJava UP-TO-DATE
+> Task :compileTestJava UP-TO-DATE
+> Task :processTestResources UP-TO-DATE
+> Task :testClasses UP-TO-DATE
+> Task :compileIntegrationTestJava UP-TO-DATE
+> Task :processIntegrationTestResources NO-SOURCE
+> Task :integrationTestClasses UP-TO-DATE
+> Task :checkstyleIntegrationTest UP-TO-DATE
+> Task :checkstyleMain UP-TO-DATE
+> Task :checkstyleTest UP-TO-DATE
+> Task :processTestFixturesResources NO-SOURCE
+> Task :testFixturesClasses UP-TO-DATE
+> Task :checkstyleTestFixtures UP-TO-DATE
+> Task :testFixturesJar UP-TO-DATE
+> Task :common:processTestFixturesResources NO-SOURCE
+> Task :common:testFixturesClasses UP-TO-DATE
+> Task :common:testFixturesJar UP-TO-DATE
+> Task :test UP-TO-DATE
+> Task :jacocoTestReport UP-TO-DATE
+> Task :spotbugsIntegrationTest UP-TO-DATE
+> Task :spotbugsMain UP-TO-DATE
+> Task :spotbugsTest UP-TO-DATE
+> Task :spotbugsTestFixtures UP-TO-DATE
+> Task :check
+> Task :copyJolokia UP-TO-DATE
+> Task :installDist
+> Task :copyDist
+> Task :docs:asciidoctor UP-TO-DATE
+> Task :copyDocs UP-TO-DATE
+> Task :generateReDoc NO-SOURCE
+> Task :generateSwaggerUI NO-SOURCE
+> Task :build
+
+> Task :run
+Could not start Jolokia agent: java.net.BindException: Address already in use
+WARNING: An illegal reflective access operation has occurred
+WARNING: Please consider reporting this to the maintainers of com.google.inject.internal.cglib.core.$ReflectUtils$1
+WARNING: Use --illegal-access=warn to enable warnings of further illegal reflective access operations
+WARNING: All illegal access operations will be denied in a future release
+INFO  [main] 2023-04-13 13:13:35,939 YAMLSidecarConfiguration.java:211 - Reading configuration from file://${HOME}/workspace/cassandra-sidecar-server/conf/sidecar.yaml
+INFO  [main] 2023-04-13 13:13:35,997 CQLSessionProvider.java:63 - Connecting to localhost on port 9042
+INFO  [main] 2023-04-13 13:13:36,001 CQLSessionProvider.java:63 - Connecting to localhost2 on port 9042
+INFO  [main] 2023-04-13 13:13:36,014 CQLSessionProvider.java:63 - Connecting to localhost3 on port 9042
+INFO  [main] 2023-04-13 13:13:36,030 CacheFactory.java:83 - Building SSTable Import Cache with expireAfterAccess=PT2H, maxSize=10000
+ _____                               _              _____ _     _                     
+/  __ \                             | |            /  ___(_)   | |                    
+| /  \/ __ _ ___ ___  __ _ _ __   __| |_ __ __ _   \ `--. _  __| | ___  ___ __ _ _ __ 
+| |    / _` / __/ __|/ _` | '_ \ / _` | '__/ _` |   `--. \ |/ _` |/ _ \/ __/ _` | '__|
+| \__/\ (_| \__ \__ \ (_| | | | | (_| | | | (_| |  /\__/ / | (_| |  __/ (_| (_| | |   
+ \____/\__,_|___/___/\__,_|_| |_|\__,_|_|  \__,_|  \____/|_|\__,_|\___|\___\__,_|_|
+                                                                                      
+                                                                                      
+INFO  [main] 2023-04-13 13:13:36,229 CassandraSidecarDaemon.java:75 - Starting Cassandra Sidecar on 0.0.0.0:9043
+```
+
+There we have it, Cassandra Sidecar is now running and connected to all 3 Cassandra nodes on my local machine.
+
+### Step 3: Run the Sample Job
+
+To be able to run the [Sample Job](./src/main/java/org/apache/cassandra/spark/example/SampleCassandraJob.java), you
+need to create the keyspace and table used for the test.
+
+Connect to your local Cassandra cluster using CCM:
+
+```shell
+ccm node1 cqlsh
+```
+
+Then run the following command to create the keyspace:
+
+```cassandraql
+CREATE KEYSPACE spark_test WITH replication = {'class': 'NetworkTopologyStrategy', 'datacenter1': '3'}
+                            AND durable_writes = true;
+```
+
+Then run the following command to create the table:
+
+```cassandraql
+CREATE TABLE spark_test.test
+(
+    id     BIGINT PRIMARY KEY,
+    course BLOB,
+    marks  BIGINT
+);
+```
+
+The Cassandra Spark Bulk Analytics project depends on the Sidecar client. Since the Sidecar dependencies for the
+Analytics have not yet been published to maven central, we need to build them locally:
+
+```shell
+cd ${SIDECAR_REPOSITORY_HOME}
+./gradlew -Pversion=1.0.0-local :common:publishToMavenLocal :client:publishToMavenLocal :vertx-client:publishToMavenLocal :vertx-client-shaded:publishToMavenLocal
+```
+
+We use the suffix `-local` to denote that these artifacts have been built locally.
+
+Finally, we are ready to run the example spark job:
+
+```shell
+cd ${ANALYTICS_REPOSITORY_HOME}
+./gradlew :cassandra-analytics-core-example:run
+```
+
+## Tear down
+
+Stop the Cassandra Sidecar project and tear down the ccm cluster
+
+```shell
+ccm remove test
+```
diff --git a/cassandra-analytics-core-example/build.gradle b/cassandra-analytics-core-example/build.gradle
new file mode 100644
index 0000000..93bce75
--- /dev/null
+++ b/cassandra-analytics-core-example/build.gradle
@@ -0,0 +1,56 @@
+/*
+ * 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.
+ */
+
+plugins {
+    id 'java'
+    id 'application'
+}
+
+repositories {
+    mavenCentral()
+}
+
+dependencies {
+    implementation(project(':cassandra-analytics-core'))
+    implementation(group: 'net.java.dev.jna', name: 'jna', version: '5.9.0')
+    implementation(group: 'org.apache.spark', name: 'spark-core_2.12', version: '3.2.1')
+    implementation(group: 'org.apache.spark', name: 'spark-sql_2.12', version: '3.2.1')
+}
+
+application {
+    mainClass = 'org.apache.cassandra.spark.example.SampleCassandraJob'
+    applicationDefaultJvmArgs = ["-Dfile.encoding=UTF-8",
+                                 "-Djdk.attach.allowAttachSelf=true",
+                                 "--add-exports", "java.base/jdk.internal.misc=ALL-UNNAMED",
+                                 "--add-exports", "java.base/jdk.internal.ref=ALL-UNNAMED",
+                                 "--add-exports", "java.base/sun.nio.ch=ALL-UNNAMED",
+                                 "--add-exports", "java.management.rmi/com.sun.jmx.remote.internal.rmi=ALL-UNNAMED",
+                                 "--add-exports", "java.rmi/sun.rmi.registry=ALL-UNNAMED",
+                                 "--add-exports", "java.rmi/sun.rmi.server=ALL-UNNAMED",
+                                 "--add-exports", "java.sql/java.sql=ALL-UNNAMED",
+                                 "--add-opens", "java.base/java.lang.module=ALL-UNNAMED",
+                                 "--add-opens", "java.base/jdk.internal.loader=ALL-UNNAMED",
+                                 "--add-opens", "java.base/jdk.internal.ref=ALL-UNNAMED",
+                                 "--add-opens", "java.base/jdk.internal.reflect=ALL-UNNAMED",
+                                 "--add-opens", "java.base/jdk.internal.math=ALL-UNNAMED",
+                                 "--add-opens", "java.base/jdk.internal.module=ALL-UNNAMED",
+                                 "--add-opens", "java.base/jdk.internal.util.jar=ALL-UNNAMED",
+                                 "--add-opens", "jdk.management/com.sun.management.internal=ALL-UNNAMED"]
+
+}
diff --git a/cassandra-analytics-core-example/src/main/java/org/apache/cassandra/spark/example/SampleCassandraJob.java b/cassandra-analytics-core-example/src/main/java/org/apache/cassandra/spark/example/SampleCassandraJob.java
new file mode 100644
index 0000000..4e21749
--- /dev/null
+++ b/cassandra-analytics-core-example/src/main/java/org/apache/cassandra/spark/example/SampleCassandraJob.java
@@ -0,0 +1,221 @@
+/*
+ * 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.cassandra.spark.example;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import java.util.stream.LongStream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.spark.KryoRegister;
+import org.apache.cassandra.spark.bulkwriter.BulkSparkConf;
+import org.apache.spark.SparkConf;
+import org.apache.spark.SparkContext;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.api.java.function.Function2;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.RowFactory;
+import org.apache.spark.sql.SQLContext;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.types.StructType;
+
+import static org.apache.spark.sql.types.DataTypes.BinaryType;
+import static org.apache.spark.sql.types.DataTypes.LongType;
+
+/**
+ * Example showcasing the Cassandra Spark Analytics write and read capabilities
+ * <p>
+ * Prepare your environment by creating the following keyspace and table
+ * <p>
+ * Schema for the {@code keyspace}:
+ * <pre>
+ *     CREATE KEYSPACE spark_test WITH replication = {'class': 'NetworkTopologyStrategy', 'datacenter1': '3'}
+ *     AND durable_writes = true;
+ * </pre>
+ * <p>
+ * Schema for the {@code table}:
+ * <pre>
+ *     CREATE TABLE spark_test.test (
+ *         id BIGINT PRIMARY KEY,
+ *         course BLOB,
+ *         marks BIGINT
+ *     );
+ * </pre>
+ */
+public final class SampleCassandraJob
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(SampleCassandraJob.class);
+
+    private SampleCassandraJob()
+    {
+        throw new IllegalStateException(getClass() + " is static utility class and shall not be instantiated");
+    }
+
+    public static void main(String[] args)
+    {
+        LOGGER.info("Starting Spark job with args={}", Arrays.toString(args));
+
+        SparkConf sparkConf = new SparkConf().setAppName("Sample Spark Cassandra Bulk Reader Job")
+                                             .set("spark.master", "local[8]");
+
+        // Add SBW-specific settings
+        // TODO: Simplify setting up spark conf
+        BulkSparkConf.setupSparkConf(sparkConf, true);
+        KryoRegister.setup(sparkConf);
+
+        SparkSession spark = SparkSession
+                             .builder()
+                             .config(sparkConf)
+                             .getOrCreate();
+        SparkContext sc = spark.sparkContext();
+        SQLContext sql = spark.sqlContext();
+        LOGGER.info("Spark Conf: " + sparkConf.toDebugString());
+        int rowCount = 10_000;
+
+        try
+        {
+            Dataset<Row> written = write(rowCount, sparkConf, sql, sc);
+            Dataset<Row> read = read(rowCount, sparkConf, sql, sc);
+
+            checkSmallDataFrameEquality(written, read);
+            LOGGER.info("Finished Spark job, shutting down...");
+            sc.stop();
+        }
+        catch (Throwable throwable)
+        {
+            LOGGER.error("Unexpected exception executing Spark job", throwable);
+            try
+            {
+                sc.stop();
+            }
+            catch (Throwable ignored)
+            {
+            }
+        }
+    }
+
+    private static Dataset<Row> write(long rowCount, SparkConf sparkConf, SQLContext sql, SparkContext sc)
+    {
+        StructType schema = new StructType()
+                            .add("id", LongType, false)
+                            .add("course", BinaryType, false)
+                            .add("marks", LongType, false);
+
+        JavaSparkContext javaSparkContext = JavaSparkContext.fromSparkContext(sc);
+        int parallelism = sc.defaultParallelism();
+        JavaRDD<Row> rows = genDataset(javaSparkContext, rowCount, parallelism);
+        Dataset<Row> df = sql.createDataFrame(rows, schema);
+
+        df.write()
+          .format("org.apache.cassandra.spark.sparksql.CassandraDataSink")
+          .option("sidecar_instances", "localhost,localhost2,localhost3")
+          .option("keyspace", "spark_test")
+          .option("table", "test")
+          .option("local_dc", "datacenter1")
+          .option("bulk_writer_cl", "LOCAL_QUORUM")
+          .option("number_splits", "-1")
+          .mode("append")
+          .save();
+        return df;
+    }
+
+    private static Dataset<Row> read(int expectedRowCount, SparkConf sparkConf, SQLContext sql, SparkContext sc)
+    {
+        int coresPerExecutor = sparkConf.getInt("spark.executor.cores", 1);
+        int numExecutors = sparkConf.getInt("spark.dynamicAllocation.maxExecutors", sparkConf.getInt("spark.executor.instances", 1));
+        int numCores = coresPerExecutor * numExecutors;
+
+        Dataset<Row> df = sql.read().format("org.apache.cassandra.spark.sparksql.CassandraDataSource")
+                             .option("sidecar_instances", "localhost,localhost2,localhost3")
+                             .option("keyspace", "spark_test")
+                             .option("table", "test")
+                             .option("DC", "datacenter1")
+                             .option("snapshotName", UUID.randomUUID().toString())
+                             .option("createSnapshot", "true")
+                             .option("defaultParallelism", sc.defaultParallelism())
+                             .option("numCores", numCores)
+                             .option("sizing", "default")
+                             .load();
+
+        long count = df.count();
+        LOGGER.info("Found {} records", count);
+
+        if (count != expectedRowCount)
+        {
+            LOGGER.error("Expected {} records but found {} records", expectedRowCount, count);
+            return null;
+        }
+        return df;
+    }
+
+    private static void checkSmallDataFrameEquality(Dataset<Row> expected, Dataset<Row> actual)
+    {
+        if (actual == null)
+        {
+            throw new NullPointerException("actual dataframe is null");
+        }
+        if (!actual.exceptAll(expected).isEmpty())
+        {
+            throw new IllegalStateException("The content of the dataframes differs");
+        }
+    }
+
+    private static JavaRDD<Row> genDataset(JavaSparkContext sc, long records, Integer parallelism)
+    {
+        long recordsPerPartition = records / parallelism;
+        long remainder = records - (recordsPerPartition * parallelism);
+        List<Integer> seq = IntStream.range(0, parallelism).boxed().collect(Collectors.toList());
+        JavaRDD<Row> dataset = sc.parallelize(seq, parallelism).mapPartitionsWithIndex(
+                (Function2<Integer, Iterator<Integer>, Iterator<Row>>) (index, integerIterator) -> {
+                    long firstRecordNumber = index * recordsPerPartition;
+                    long recordsToGenerate = index.equals(parallelism) ? remainder : recordsPerPartition;
+                    java.util.Iterator<Row> rows = LongStream.range(0, recordsToGenerate).mapToObj(offset -> {
+                        long recordNumber = firstRecordNumber + offset;
+                        String courseNameString = String.valueOf(recordNumber);
+                        Integer courseNameStringLen = courseNameString.length();
+                        Integer courseNameMultiplier = 1000 / courseNameStringLen;
+                        byte[] courseName = dupStringAsBytes(courseNameString, courseNameMultiplier);
+                        return RowFactory.create(recordNumber, courseName, recordNumber);
+                    }).iterator();
+                    return rows;
+                }, false);
+        return dataset;
+    }
+
+    private static byte[] dupStringAsBytes(String string, Integer times)
+    {
+        byte[] stringBytes = string.getBytes();
+        ByteBuffer buffer = ByteBuffer.allocate(stringBytes.length * times);
+        for (int time = 0; time < times; time++)
+        {
+            buffer.put(stringBytes);
+        }
+        return buffer.array();
+    }
+}
diff --git a/cassandra-analytics-core/build.gradle b/cassandra-analytics-core/build.gradle
new file mode 100644
index 0000000..77cae04
--- /dev/null
+++ b/cassandra-analytics-core/build.gradle
@@ -0,0 +1,179 @@
+/*
+ * 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(':cassandra-analytics-core') {
+    apply(plugin: 'java-library')
+    apply(plugin: 'jacoco')
+    apply(plugin: 'maven-publish')
+
+    publishing {
+        publications {
+            maven(MavenPublication) {
+                from components.java
+                groupId project.group
+                artifactId "${archivesBaseName}"
+                version version
+            }
+        }
+    }
+
+    configurations {
+        fourzero {
+            description = 'Bridge to Cassandra 4.0 with all of its dependencies'
+        }
+        all {
+            resolutionStrategy {
+                force("com.fasterxml.jackson.core:jackson-databind:${jacksonVersion}")
+            }
+        }
+    }
+
+    dependencies {
+        compileOnly(project(':cassandra-bridge'))
+
+        fourzero(project(path: ':cassandra-four-zero', configuration: 'shadow'))
+
+        // Including newer JNA which works with "Modern" Linux/GLIBC.
+        // End users can exclude this and include their own if necessary.
+        runtimeOnly(group: 'net.java.dev.jna', name: 'jna', version: "${jnaVersion}")
+        runtimeOnly(group: 'net.java.dev.jna', name: 'jna-platform', version: "${jnaVersion}")
+
+        // This dependency needs to be installed locally by building the Cassandra Sidecar project using the following
+        // commands:
+        //  ./gradlew -Pversion=1.0.0-local :common:publishToMavenLocal :client:publishToMavenLocal :vertx-client:publishToMavenLocal :vertx-client-shaded:publishToMavenLocal
+        implementation(group: 'org.apache.cassandra.sidecar', name: 'vertx-client-all', version: "${sidecarVersion}") {
+            exclude(group: 'com.fasterxml.jackson.core', module: 'jackson-annotations') // use provided annotations from spark
+        }
+
+        if ("${scalaMajorVersion}" == "2.11")
+        {
+            implementation(group: 'org.scala-lang.modules', name: "scala-java8-compat_2.11", version: '1.0.1', transitive: false)
+        }
+
+        compileOnly(group: "${sparkGroupId}", name: "spark-core_${scalaMajorVersion}", version: "${project.rootProject.sparkVersion}")
+        compileOnly(group: "${sparkGroupId}", name: "spark-sql_${scalaMajorVersion}", version: "${project.rootProject.sparkVersion}")
+
+        testImplementation(group: 'com.google.guava', name: 'guava', version: '19.0')
+        testImplementation(group: 'junit', name: 'junit', version: "${project.rootProject.junitVersion}")
+        testImplementation(group: 'org.quicktheories', name: 'quicktheories', version: "${project.rootProject.quickTheoriesVersion}")
+        testImplementation(group: 'org.slf4j', name: 'slf4j-simple', version: '1.7.26')
+        testImplementation(group: 'org.mockito', name: 'mockito-core', version: "${project.rootProject.mockitoVersion}")
+        testImplementation(group: 'org.mockito', name: 'mockito-inline', version: "${project.rootProject.mockitoVersion}")
+        testImplementation(group: "${sparkGroupId}", name: "spark-core_${scalaMajorVersion}", version: "${project.rootProject.sparkVersion}")
+        testImplementation(group: "${sparkGroupId}", name: "spark-sql_${scalaMajorVersion}", version: "${project.rootProject.sparkVersion}")
+        testImplementation(group: 'org.hamcrest', name: 'hamcrest-all', version: '1.3')
+        testImplementation(group: 'org.openjdk.jmh', name: 'jmh-core', version: '1.33')
+        testImplementation(group: 'org.apache.httpcomponents', name: 'httpclient', version: '4.5.13')
+        testImplementation(group: 'com.github.luben', name: 'zstd-jni', version: '1.5.0-4')
+        testImplementation(group: 'org.xerial.snappy', name: 'snappy-java', version: '1.1.8.4')
+
+        testImplementation(group: 'com.github.luben', name: 'zstd-jni', version: '1.5.0-4')
+        testCompileOnly(group: "${sparkGroupId}", name: "spark-core_${scalaMajorVersion}", version: "${project.rootProject.sparkVersion}")
+        testCompileOnly(group: "${sparkGroupId}", name: "spark-sql_${scalaMajorVersion}", version: "${project.rootProject.sparkVersion}")
+
+        testRuntimeOnly(group: 'net.java.dev.jna', name: 'jna', version: "${jnaVersion}")
+        testRuntimeOnly(group: 'net.java.dev.jna', name: 'jna-platform', version: "${jnaVersion}")
+
+        testImplementation(project(':cassandra-bridge'))
+    }
+
+    def utils = ['org/apache/cassandra/spark/bulkwriter/MockBulkWriterContext.class',
+                 'org/apache/cassandra/spark/bulkwriter/MockBulkWriterContext$CommitResultSupplier.class',
+                 'org/apache/cassandra/spark/bulkwriter/RingUtils.class',
+                 'org/apache/cassandra/spark/bulkwriter/TableSchemaTestCommon.class',
+                 'org/apache/cassandra/spark/bulkwriter/TableSchemaTestCommon$MockTableInfoProvider.class',
+                 'org/apache/cassandra/spark/bulkwriter/TableSchemaTestCommon$MockTableSchemaBuilder.class']
+
+    jar {
+        dependsOn(configurations.fourzero)
+
+        writeBuildVersion(version, project.projectDir)
+
+        exclude('META-INF/*')
+        from(project(':cassandra-bridge').sourceSets.main.output)
+        from(project(':cassandra-analytics-core').sourceSets.test.output) {
+            include(utils)
+        }
+        from(project(':cassandra-analytics-core').sourceSets.test.output) {
+            exclude(utils)
+            into('tests')
+        }
+        into('bridges') {
+            from(configurations.fourzero.singleFile)
+        }
+    }
+
+    test {
+        minHeapSize = '1024m'
+        maxHeapSize = '3072m'
+        maxParallelForks = Math.max(Runtime.runtime.availableProcessors() * 2, 8)
+        forkEvery = 1  // Enables different end-to-end test classes use Spark contexts with different configurations
+
+        // Make it so unit tests run on a JAr with Cassandra bridge implementations built in
+        dependsOn(tasks.jar)
+        classpath = project.sourceSets.test.output + configurations.testRuntimeClasspath + files(jar.archiveFile)
+        useJUnit()
+    }
+
+    /* Start: JaCoCo check */
+    jacoco {
+        toolVersion = '0.8.4'
+    }
+    task jacocoMerge(type: JacocoMerge) {
+        destinationFile = file("$buildDir/jacoco/mergedFile.exec")
+        executionData = files("$buildDir/jacoco/test.exec")
+    }
+    jacocoTestReport {
+        dependsOn(tasks.jacocoMerge)
+        executionData(tasks.jacocoMerge.executionData)
+        reports {
+            html.required = true
+            html.destination(file("$buildDir/reports/jacocoHtml"))
+            xml.required = true
+            xml.destination(file("$buildDir/reports/jacocoXml"))
+        }
+    }
+    jacocoTestCoverageVerification {
+        dependsOn(tasks.jacocoMerge)
+        executionData(tasks.jacocoMerge.executionData)
+        violationRules {
+            rule {
+                limit {
+                    minimum = 0.5
+                }
+            }
+        }
+    }
+    check {
+        dependsOn(tasks.jacocoTestCoverageVerification)
+        finalizedBy(tasks.jacocoTestReport)
+    }
+    /* End: JaCoCo check */
+}
+
+private void writeBuildVersion(version, projectDir) {
+    def prop = new Properties()
+    prop.setProperty('build-version', "${version}")
+    def hash = System.getenv('GIT_HASH') ?: 'unknown'
+    prop.setProperty('build-rev', hash)
+    def propFile = new File("${projectDir}/src/main/resources/cassandra-analytics-build.properties")
+    propFile.createNewFile()
+    prop.store(propFile.newWriter(), null)
+    logger.info("Wrote to file={}, version={}, hash={}", propFile, version, hash)
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/bridge/CassandraBridgeFactory.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/bridge/CassandraBridgeFactory.java
new file mode 100644
index 0000000..7e2cd51
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/bridge/CassandraBridgeFactory.java
@@ -0,0 +1,135 @@
+/*
+ * 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.cassandra.bridge;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+import java.net.URL;
+import java.nio.file.Files;
+import java.util.Arrays;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.io.FileUtils;
+
+import org.jetbrains.annotations.NotNull;
+
+public final class CassandraBridgeFactory
+{
+    private static final Map<String, CassandraBridge> CASSANDRA_BRIDGES = new ConcurrentHashMap<>(CassandraVersion.values().length);
+
+    private CassandraBridgeFactory()
+    {
+        throw new IllegalStateException(getClass() + " is static utility class and shall not be instantiated");
+    }
+
+    @NotNull
+    public static CassandraVersion getCassandraVersion(@NotNull String version)
+    {
+        CassandraVersionFeatures features = CassandraVersionFeatures.cassandraVersionFeaturesFromCassandraVersion(version);
+        Preconditions.checkArgument(features != null, "Cassandra version " + version + " is not supported");
+        return getCassandraVersion(features);
+    }
+
+    @NotNull
+    public static CassandraVersion getCassandraVersion(@NotNull CassandraVersionFeatures features)
+    {
+        Optional<CassandraVersion> version = Arrays.stream(CassandraVersion.values())
+                .filter(value -> value.versionNumber() == features.getMajorVersion())
+                .findAny();
+        Preconditions.checkArgument(version.isPresent(), "Cassandra features " + features + " are not supported");
+        return version.get();
+    }
+
+    @NotNull
+    public static CassandraBridge get(@NotNull String version)
+    {
+        return get(getCassandraVersion(version));
+    }
+
+    @NotNull
+    public static CassandraBridge get(@NotNull CassandraVersionFeatures features)
+    {
+        return get(getCassandraVersion(features));
+    }
+
+    @NotNull
+    public static CassandraBridge get(@NotNull CassandraVersion version)
+    {
+        String jarBaseName = version.jarBaseName();
+        Preconditions.checkNotNull(jarBaseName, "Cassandra version " + version + " is not supported");
+        return CASSANDRA_BRIDGES.computeIfAbsent(jarBaseName, CassandraBridgeFactory::create);
+    }
+
+    /**
+     * Ensures that every supported Cassandra version has a corresponding Cassandra bridge implementation embedded
+     * into this library's binary as a separate JAR file in the {@code bridges} directory, fails fast otherwise
+     *
+     * @throws IllegalStateException If a Cassandra bridge implementation is missing for a supported Cassandra version
+     */
+    public static void validateBridges()
+    {
+        for (CassandraVersion version : CassandraVersion.implementedVersions())
+        {
+            String jarBaseName = version.jarBaseName();
+            String name = resourceName(jarBaseName);
+            URL locator = CassandraBridgeFactory.class.getResource(name);
+            if (locator == null)
+            {
+                throw new IllegalStateException("Missing Cassandra bridge implementation for version " + version);
+            }
+        }
+    }
+
+    @NotNull
+    private static String resourceName(@NotNull String label)
+    {
+        return "/bridges/" + label + ".jar";
+    }
+
+    @NotNull
+    @SuppressWarnings("unchecked")
+    private static CassandraBridge create(@NotNull String label)
+    {
+        try
+        {
+            String name = resourceName(label);
+            InputStream contents = CassandraBridgeFactory.class.getResourceAsStream(name);
+            File jar = Files.createTempFile(null, ".jar").toFile();
+            FileUtils.copyInputStreamToFile(contents, jar);
+
+            ClassLoader loader = new PostDelegationClassLoader(jar, Thread.currentThread().getContextClassLoader());
+            Class<CassandraBridge> bridge = (Class<CassandraBridge>) loader.loadClass("org.apache.cassandra.bridge.CassandraBridgeImplementation");
+            Constructor<CassandraBridge> constructor = bridge.getConstructor();
+            return constructor.newInstance();
+
+        }
+        catch (IOException | ClassNotFoundException | NoSuchMethodException | InstantiationException
+             | IllegalAccessException | InvocationTargetException exception)
+        {
+            throw new RuntimeException("Failed to create Cassandra bridge for label " + label, exception);
+        }
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/bridge/CassandraVersionFeatures.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/bridge/CassandraVersionFeatures.java
new file mode 100644
index 0000000..089f574
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/bridge/CassandraVersionFeatures.java
@@ -0,0 +1,198 @@
+/*
+ * 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.cassandra.bridge;
+
+import java.util.Objects;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+public class CassandraVersionFeatures implements Comparable<CassandraVersionFeatures>
+{
+    private static final Pattern VERSION_PATTERN_3        = Pattern.compile("(?:.+-)?([0-9]+)\\.([0-9]+)\\.([0-9]+)([a-zA-Z0-9-]*)");
+    private static final Pattern VERSION_PATTERN_4        = Pattern.compile("(?:.+-)?([0-9]+)\\.([0-9]+)\\.([0-9]+)\\.([0-9]+)[a-zA-Z0-9-]*");
+    private static final Pattern VERSION_PATTERN_SNAPSHOT = Pattern.compile("(?:.+-)?([0-9]+)\\.([0-9]+)-(SNAPSHOT)$");
+
+    protected final int majorVersion;
+    protected final int minorVersion;
+    protected final String suffix;
+
+    public CassandraVersionFeatures(int majorVersion, int minorVersion, @Nullable String suffix)
+    {
+        this.majorVersion = majorVersion;
+        this.minorVersion = minorVersion;
+        this.suffix = suffix;
+    }
+
+    /**
+     * Given a cassandra version string (e.g. cassandra-2.0.14-v3) parse it into a CassandraVersionFeatures object
+     *
+     * @param cassandraVersion the string representing the cassandra version
+     * @return an instance representing the parsed values from the version string
+     */
+    public static CassandraVersionFeatures cassandraVersionFeaturesFromCassandraVersion(
+            @NotNull String cassandraVersion)
+    {
+        String versionCode = getCassandraVersionCode(cassandraVersion);
+        String minorVersionCode = getCassandraMinorVersionCode(cassandraVersion);
+        String versionSuffix = getCassandraVersionSuffix(cassandraVersion);
+
+        return new CassandraVersionFeatures(Integer.parseInt(versionCode), Integer.parseInt(minorVersionCode), versionSuffix);
+    }
+
+    // E.g if cassandra version = cassandra-1.2.11-v1, we return 12;
+    //  or if cassandra version = cassandra-4.0-SNAPSHOT, we return 40
+    private static String getCassandraVersionCode(String cassandraVersion)
+    {
+        Matcher matcher = matchVersion(cassandraVersion);
+
+        return matcher.group(1) + matcher.group(2);
+    }
+
+    // E.g if cassandra version = cassandra-1.2.11-v1, we return 11;
+    //  or if cassandra version = cassandra-4.0-SNAPSHOT, we return 0
+    private static String getCassandraMinorVersionCode(String cassandraVersion)
+    {
+        Matcher matcher = matchVersion(cassandraVersion);
+        if (matchesSnapshot(matcher.group(3)))
+        {
+            return "0";
+        }
+
+        return matcher.group(3);
+    }
+
+    // E.g if cassandra version = cassandra-1.2.11-v1, we return -v1;
+    //  or if cassandra version = cassandra-1.2.11.2-tag, we return 2;
+    //  or if cassandra version = cassandra-4.0-SNAPSHOT, we return SNAPSHOT
+    private static String getCassandraVersionSuffix(String cassandraVersion)
+    {
+        Matcher matcher = matchVersion(cassandraVersion);
+        if (matchesSnapshot(matcher.group(3)) || matchesSnapshot(matcher.group(4)))
+        {
+            return "SNAPSHOT";
+        }
+
+        return matcher.group(4);
+    }
+
+    private static boolean matchesSnapshot(String snapshot)
+    {
+        return "SNAPSHOT".equals(snapshot) || "-SNAPSHOT".equals(snapshot);
+    }
+
+    /**
+     * Returns a matched matcher using VERSION_PATTERN; throws if no match
+     */
+    private static Matcher matchVersion(String cassandraVersion)
+    {
+        Matcher matcher = VERSION_PATTERN_4.matcher(cassandraVersion);
+
+        if (!matcher.find())
+        {
+            matcher = VERSION_PATTERN_3.matcher(cassandraVersion);
+
+            if (!matcher.find())
+            {
+                matcher = VERSION_PATTERN_SNAPSHOT.matcher(cassandraVersion);
+                if (!matcher.find())
+                {
+                    throw new RuntimeException("cassandraVersion does not match version pattern, pattern=" + VERSION_PATTERN_3
+                                                                                            + ", version=" + cassandraVersion);
+                }
+            }
+        }
+        return matcher;
+    }
+
+    public int getMajorVersion()
+    {
+        return majorVersion;
+    }
+
+    public int getMinorVersion()
+    {
+        return minorVersion;
+    }
+
+    public String getSuffix()
+    {
+        return suffix;
+    }
+
+    @Override
+    public int compareTo(@NotNull CassandraVersionFeatures that)
+    {
+        int difference = this.getMajorVersion() - that.getMajorVersion();
+        if (difference != 0)
+        {
+            return difference;
+        }
+
+        difference = this.getMinorVersion() - that.getMinorVersion();
+        if (difference != 0)
+        {
+            return difference;
+        }
+
+        // Try to treat suffix as micro version
+        try
+        {
+            return Integer.parseInt(getSuffix()) - Integer.parseInt(that.getSuffix());
+        }
+        catch (NumberFormatException exception)
+        {
+            // One with smallest suffix will be smallest
+            int thisLength = getSuffix() != null ? getSuffix().length() : 0;
+            int thatLength = that.getSuffix() != null ? that.getSuffix().length() : 0;
+            return thisLength - thatLength;
+        }
+    }
+
+    @Override
+    public boolean equals(@Nullable Object other)
+    {
+        if (other == null || !(other instanceof CassandraVersionFeatures))
+        {
+            return false;
+        }
+
+        CassandraVersionFeatures that = (CassandraVersionFeatures) other;
+        return Objects.equals(this.majorVersion, that.majorVersion)
+            && Objects.equals(this.minorVersion, that.minorVersion)
+            && Objects.equals(this.suffix, that.suffix);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Objects.hash(majorVersion, minorVersion, suffix);
+    }
+
+    @Override
+    @NotNull
+    public String toString()
+    {
+        return String.format("CassandraVersionFeatures{majorVersion=%d, minorVersion=%d, suffix='%s'}",
+                             majorVersion, minorVersion, suffix);
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/bridge/PostDelegationClassLoader.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/bridge/PostDelegationClassLoader.java
new file mode 100644
index 0000000..c1e26a7
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/bridge/PostDelegationClassLoader.java
@@ -0,0 +1,71 @@
+/*
+ * 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.cassandra.bridge;
+
+import java.io.File;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.net.URLClassLoader;
+
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * This custom implementation of a {@link ClassLoader} enables deferred execution-time loading of a particular version
+ * of class hierarchy from one of many embedded the {@code cassandra-all} library JARs. It first attempts to load any
+ * requested class from the extracted JAR, and resorts to using the parent class loader when the the class is not there.
+ * This behavior is opposite to the one of standard {@link URLClassLoader}, which invokes its parent class loader first.
+ */
+public class PostDelegationClassLoader extends URLClassLoader
+{
+    public PostDelegationClassLoader(@NotNull File jar, @Nullable ClassLoader parent) throws MalformedURLException
+    {
+        super(new URL[]{jar.toURI().toURL()}, parent);
+    }
+
+    @Override
+    @Nullable
+    protected synchronized Class<?> loadClass(@Nullable String name, boolean resolve) throws ClassNotFoundException
+    {
+        // First, check if the class has already been loaded
+        Class<?> type = findLoadedClass(name);
+        if (type == null)
+        {
+            try
+            {
+               type = findClass(name);
+            }
+            catch (ClassNotFoundException | SecurityException | LinkageError exception)
+            {
+                // ClassNotFoundException thrown if class not found
+            }
+            if (type == null)
+            {
+                // If not found, then invoke findClass in order to find the class
+                type = super.loadClass(name, false);
+            }
+        }
+        if (resolve)
+        {
+            resolveClass(type);
+        }
+        return type;
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/clients/ExecutorHolder.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/clients/ExecutorHolder.java
new file mode 100644
index 0000000..dbb4df1
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/clients/ExecutorHolder.java
@@ -0,0 +1,42 @@
+/*
+ * 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.cassandra.clients;
+
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+/**
+ * A class that holds Executors used during the Cassandra Analytics execution
+ */
+public final class ExecutorHolder
+{
+    // TODO: Make the size configurable
+    public static final ExecutorService EXECUTOR_SERVICE = Executors.newFixedThreadPool(64,
+            new ThreadFactoryBuilder().setNameFormat("cassandra-analytics-%d")
+                                      .setDaemon(true)
+                                      .build());
+
+    private ExecutorHolder()
+    {
+        throw new IllegalStateException(getClass() + " is static utility class and shall not be instantiated");
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/clients/Sidecar.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/clients/Sidecar.java
new file mode 100644
index 0000000..71d8f43
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/clients/Sidecar.java
@@ -0,0 +1,405 @@
+/*
+ * 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.cassandra.clients;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import o.a.c.sidecar.client.shaded.io.vertx.core.Vertx;
+import o.a.c.sidecar.client.shaded.io.vertx.core.VertxOptions;
+import org.apache.cassandra.sidecar.client.HttpClientConfig;
+import org.apache.cassandra.sidecar.client.SidecarClient;
+import org.apache.cassandra.sidecar.client.SidecarConfig;
+import org.apache.cassandra.sidecar.client.SidecarInstance;
+import org.apache.cassandra.sidecar.client.SidecarInstancesProvider;
+import org.apache.cassandra.sidecar.client.VertxHttpClient;
+import org.apache.cassandra.sidecar.client.VertxRequestExecutor;
+import org.apache.cassandra.sidecar.client.retry.ExponentialBackoffRetryPolicy;
+import org.apache.cassandra.sidecar.client.retry.RetryPolicy;
+import org.apache.cassandra.sidecar.common.NodeSettings;
+import org.apache.cassandra.spark.bulkwriter.BulkSparkConf;
+import org.apache.cassandra.spark.data.FileType;
+import org.apache.cassandra.spark.utils.BuildInfo;
+import org.apache.cassandra.spark.utils.MapUtils;
+
+import static org.apache.cassandra.spark.utils.Properties.DEFAULT_CHUNK_BUFFER_OVERRIDE;
+import static org.apache.cassandra.spark.utils.Properties.DEFAULT_CHUNK_BUFFER_SIZE;
+import static org.apache.cassandra.spark.utils.Properties.DEFAULT_MAX_BUFFER_OVERRIDE;
+import static org.apache.cassandra.spark.utils.Properties.DEFAULT_MAX_BUFFER_SIZE;
+import static org.apache.cassandra.spark.utils.Properties.DEFAULT_MAX_MILLIS_TO_SLEEP;
+import static org.apache.cassandra.spark.utils.Properties.DEFAULT_MAX_POOL_SIZE;
+import static org.apache.cassandra.spark.utils.Properties.DEFAULT_MAX_RETRIES;
+import static org.apache.cassandra.spark.utils.Properties.DEFAULT_MILLIS_TO_SLEEP;
+import static org.apache.cassandra.spark.utils.Properties.DEFAULT_SIDECAR_PORT;
+import static org.apache.cassandra.spark.utils.Properties.DEFAULT_TIMEOUT_SECONDS;
+
+/**
+ * A helper class that encapsulates configuration for the Spark Bulk Reader and Writer and helper methods to build the
+ * {@link SidecarClient}
+ */
+public final class Sidecar
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(Sidecar.class);
+
+    private Sidecar()
+    {
+        throw new IllegalStateException(getClass() + " is static utility class and shall not be instantiated");
+    }
+
+    public static SidecarClient from(SidecarInstancesProvider sidecarInstancesProvider,
+                                     ClientConfig config,
+                                     SslConfig sslConfig) throws IOException
+    {
+        Vertx vertx = Vertx.vertx(new VertxOptions().setUseDaemonThread(true).setWorkerPoolSize(config.maxPoolSize()));
+
+        HttpClientConfig.Builder<?> builder = new HttpClientConfig.Builder<>()
+                .ssl(false)
+                .timeoutMillis(TimeUnit.SECONDS.toMillis(config.timeoutSeconds()))
+                .idleTimeoutMillis((int) TimeUnit.SECONDS.toMillis(config.timeoutSeconds()))
+                .receiveBufferSize((int) config.chunkBufferSize())
+                .maxChunkSize((int) config.maxBufferSize())
+                .userAgent(BuildInfo.READER_USER_AGENT);
+
+        if (sslConfig != null)
+        {
+            builder = builder
+                    .ssl(true)
+                    .keyStoreInputStream(sslConfig.keyStoreInputStream())
+                    .keyStorePassword(sslConfig.keyStorePassword())
+                    .keyStoreType(sslConfig.keyStoreType())
+                    .trustStoreInputStream(sslConfig.trustStoreInputStream())
+                    .trustStorePassword(sslConfig.trustStorePassword())
+                    .trustStoreType(sslConfig.trustStoreType());
+        }
+
+        HttpClientConfig httpClientConfig = builder.build();
+
+        SidecarConfig sidecarConfig = new SidecarConfig.Builder()
+                .maxRetries(config.maxRetries())
+                .retryDelayMillis(config.millisToSleep())
+                .maxRetryDelayMillis(config.maxMillisToSleep())
+                .build();
+
+        return buildClient(sidecarConfig, vertx, httpClientConfig, sidecarInstancesProvider);
+    }
+
+    public static SidecarClient from(SidecarInstancesProvider sidecarInstancesProvider, BulkSparkConf conf)
+    {
+        Vertx vertx = Vertx.vertx(new VertxOptions().setUseDaemonThread(true)
+                                                    .setWorkerPoolSize(conf.getMaxHttpConnections()));
+
+        HttpClientConfig httpClientConfig = new HttpClientConfig.Builder<>()
+                .timeoutMillis(conf.getHttpResponseTimeoutMs())
+                .idleTimeoutMillis(conf.getHttpConnectionTimeoutMs())
+                .userAgent(BuildInfo.WRITER_USER_AGENT)
+                .keyStoreInputStream(conf.getKeyStore())
+                .keyStorePassword(conf.getKeyStorePassword())
+                .keyStoreType(conf.getKeyStoreTypeOrDefault())
+                .trustStoreInputStream(conf.getTrustStore())
+                .trustStorePassword(conf.getTrustStorePasswordOrDefault())
+                .trustStoreType(conf.getTrustStoreTypeOrDefault())
+                .ssl(conf.hasKeystoreAndKeystorePassword())
+                .build();
+
+        SidecarConfig sidecarConfig = new SidecarConfig.Builder()
+                .maxRetries(conf.getSidecarRequestRetries())
+                .retryDelayMillis(TimeUnit.SECONDS.toMillis(conf.getSidecarRequestRetryDelayInSeconds()))
+                .maxRetryDelayMillis(TimeUnit.SECONDS.toMillis(conf.getSidecarRequestMaxRetryDelayInSeconds()))
+                .build();
+
+        return buildClient(sidecarConfig, vertx, httpClientConfig, sidecarInstancesProvider);
+    }
+
+    public static SidecarClient buildClient(SidecarConfig sidecarConfig,
+                                            Vertx vertx,
+                                            HttpClientConfig httpClientConfig,
+                                            SidecarInstancesProvider clusterConfig)
+    {
+        RetryPolicy defaultRetryPolicy = new ExponentialBackoffRetryPolicy(sidecarConfig.maxRetries(),
+                                                                           sidecarConfig.retryDelayMillis(),
+                                                                           sidecarConfig.maxRetryDelayMillis());
+
+        VertxHttpClient vertxHttpClient = new VertxHttpClient(vertx, httpClientConfig);
+        VertxRequestExecutor requestExecutor = new VertxRequestExecutor(vertxHttpClient);
+        return new SidecarClient(clusterConfig, requestExecutor, sidecarConfig, defaultRetryPolicy);
+    }
+
+    public static List<NodeSettings> allNodeSettingsBlocking(BulkSparkConf conf,
+                                                             SidecarClient client,
+                                                             Set<? extends SidecarInstance> instances)
+    {
+        List<CompletableFuture<Void>> futures = new ArrayList<>();
+        List<NodeSettings> allNodeSettings = Collections.synchronizedList(new ArrayList<>());
+        for (SidecarInstance instance : instances)
+        {
+            futures.add(client.nodeSettings(instance)
+                              .exceptionally(throwable -> {
+                                  LOGGER.warn(String.format("Failed to execute node settings on instance=%s",
+                                                            instance), throwable);
+                                  return null;
+                              })
+                              .thenAccept(nodeSettings -> {
+                                  if (nodeSettings != null)
+                                  {
+                                      allNodeSettings.add(nodeSettings);
+                                  }
+                              }));
+        }
+        try
+        {
+            CompletableFuture.allOf(futures.toArray(new CompletableFuture[0]))
+                             .get(conf.getSidecarRequestMaxRetryDelayInSeconds(), TimeUnit.SECONDS);
+        }
+        catch (ExecutionException | InterruptedException exception)
+        {
+            throw new RuntimeException(exception);
+        }
+        catch (TimeoutException exception)
+        {
+            // Any futures that have already completed will have put their results into `allNodeSettings`
+            // at this point. Cancel any remaining futures and move on.
+            for (CompletableFuture<Void> future : futures)
+            {
+                future.cancel(true);
+            }
+        }
+        long successCount = allNodeSettings.size();
+        if (successCount == 0)
+        {
+            throw new RuntimeException(String.format("Unable to determine the node settings. 0/%d instances available.",
+                                                     instances.size()));
+        }
+        else if (successCount < instances.size())
+        {
+            LOGGER.debug("{}/{} instances were used to determine the node settings",
+                         successCount, instances.size());
+        }
+        return allNodeSettings;
+    }
+
+    public static final class ClientConfig
+    {
+        public static final String SIDECAR_PORT = "sidecar_port";
+        public static final String MAX_RETRIES_KEY = "maxRetries";
+        public static final String DEFAULT_MILLIS_TO_SLEEP_KEY = "defaultMillisToSleep";
+        public static final String MAX_MILLIS_TO_SLEEP_KEY = "maxMillisToSleep";
+        public static final String MAX_BUFFER_SIZE_BYTES_KEY = "maxBufferSizeBytes";
+        public static final String CHUNK_BUFFER_SIZE_BYTES_KEY = "chunkBufferSizeBytes";
+        public static final String MAX_POOL_SIZE_KEY = "maxPoolSize";
+        public static final String TIMEOUT_SECONDS_KEY = "timeoutSeconds";
+
+        private final int port;
+        private final int maxRetries;
+        private final int maxPoolSize;
+        private final int timeoutSeconds;
+        private final long millisToSleep;
+        private final long maxMillisToSleep;
+        private final long maxBufferSize;
+        private final long chunkSize;
+        private final Map<FileType, Long> maxBufferOverride;
+        private final Map<FileType, Long> chunkBufferOverride;
+
+        // CHECKSTYLE IGNORE: Constructor with many parameters
+        private ClientConfig(int port,
+                             int maxRetries,
+                             long millisToSleep,
+                             long maxMillisToSleep,
+                             long maxBufferSize,
+                             long chunkSize,
+                             int maxPoolSize,
+                             int timeoutSeconds,
+                             Map<FileType, Long> maxBufferOverride,
+                             Map<FileType, Long> chunkBufferOverride)
+        {
+            this.port = port;
+            this.maxRetries = maxRetries;
+            this.millisToSleep = millisToSleep;
+            this.maxMillisToSleep = maxMillisToSleep;
+            this.maxBufferSize = maxBufferSize;
+            this.chunkSize = chunkSize;
+            this.maxPoolSize = maxPoolSize;
+            this.timeoutSeconds = timeoutSeconds;
+            this.maxBufferOverride = maxBufferOverride;
+            this.chunkBufferOverride = chunkBufferOverride;
+        }
+
+        public int port()
+        {
+            return port;
+        }
+
+        public int maxRetries()
+        {
+            return maxRetries;
+        }
+
+        public long millisToSleep()
+        {
+            return millisToSleep;
+        }
+
+        public long maxMillisToSleep()
+        {
+            return maxMillisToSleep;
+        }
+
+        public long maxBufferSize()
+        {
+            return maxBufferSize(FileType.DATA);
+        }
+
+        public long maxBufferSize(FileType fileType)
+        {
+            return maxBufferOverride.getOrDefault(fileType, maxBufferSize);
+        }
+
+        public Map<FileType, Long> maxBufferOverride()
+        {
+            return maxBufferOverride;
+        }
+
+        public long chunkBufferSize()
+        {
+            return chunkBufferSize(FileType.DATA);
+        }
+
+        public long chunkBufferSize(FileType fileType)
+        {
+            return chunkBufferOverride.getOrDefault(fileType, chunkSize);
+        }
+
+        public Map<FileType, Long> chunkBufferOverride()
+        {
+            return chunkBufferOverride;
+        }
+
+        public int maxPoolSize()
+        {
+            return maxPoolSize;
+        }
+
+        public int timeoutSeconds()
+        {
+            return timeoutSeconds;
+        }
+
+        public static ClientConfig create()
+        {
+            return ClientConfig.create(DEFAULT_SIDECAR_PORT, DEFAULT_MAX_RETRIES, DEFAULT_MILLIS_TO_SLEEP);
+        }
+
+        public static ClientConfig create(int port)
+        {
+            return ClientConfig.create(port, DEFAULT_MAX_RETRIES, DEFAULT_MILLIS_TO_SLEEP);
+        }
+
+        public static ClientConfig create(int port, int maxRetries, long millisToSleep)
+        {
+            return ClientConfig.create(port,
+                                       maxRetries,
+                                       millisToSleep,
+                                       DEFAULT_MAX_MILLIS_TO_SLEEP,
+                                       DEFAULT_MAX_BUFFER_SIZE,
+                                       DEFAULT_CHUNK_BUFFER_SIZE,
+                                       DEFAULT_MAX_POOL_SIZE,
+                                       DEFAULT_TIMEOUT_SECONDS,
+                                       DEFAULT_MAX_BUFFER_OVERRIDE,
+                                       DEFAULT_CHUNK_BUFFER_OVERRIDE);
+        }
+
+        public static ClientConfig create(Map<String, String> options)
+        {
+            return create(MapUtils.getInt(options, SIDECAR_PORT, DEFAULT_SIDECAR_PORT),
+                          MapUtils.getInt(options, MAX_RETRIES_KEY, DEFAULT_MAX_RETRIES),
+                          MapUtils.getLong(options, DEFAULT_MILLIS_TO_SLEEP_KEY, DEFAULT_MILLIS_TO_SLEEP),
+                          MapUtils.getLong(options, MAX_MILLIS_TO_SLEEP_KEY, DEFAULT_MAX_MILLIS_TO_SLEEP),
+                          MapUtils.getLong(options, MAX_BUFFER_SIZE_BYTES_KEY, DEFAULT_MAX_BUFFER_SIZE),
+                          MapUtils.getLong(options, CHUNK_BUFFER_SIZE_BYTES_KEY, DEFAULT_CHUNK_BUFFER_SIZE),
+                          MapUtils.getInt(options, MAX_POOL_SIZE_KEY, DEFAULT_MAX_POOL_SIZE),
+                          MapUtils.getInt(options, TIMEOUT_SECONDS_KEY, DEFAULT_TIMEOUT_SECONDS),
+                          buildMaxBufferOverride(options, DEFAULT_MAX_BUFFER_OVERRIDE),
+                          buildChunkBufferOverride(options, DEFAULT_CHUNK_BUFFER_OVERRIDE)
+            );
+        }
+
+        public static Map<FileType, Long> buildMaxBufferOverride(Map<String, String> options,
+                                                                 Map<FileType, Long> defaultValue)
+        {
+            return buildOverrideMap(MAX_BUFFER_SIZE_BYTES_KEY, options, defaultValue);
+        }
+
+        public static Map<FileType, Long> buildChunkBufferOverride(Map<String, String> options,
+                                                                   Map<FileType, Long> defaultValue)
+        {
+            return buildOverrideMap(CHUNK_BUFFER_SIZE_BYTES_KEY, options, defaultValue);
+        }
+
+        private static Map<FileType, Long> buildOverrideMap(String keyPrefix,
+                                                            Map<String, String> options,
+                                                            Map<FileType, Long> defaultValue)
+        {
+            Map<FileType, Long> result = new HashMap<>(defaultValue);
+            for (FileType type : FileType.values())
+            {
+                // Override with DataSourceOptions if set, e.g. maxBufferSizeBytes_Index.db
+                String key = MapUtils.lowerCaseKey(String.format("%s_%s", keyPrefix, type.getFileSuffix()));
+                Optional.ofNullable(options.get(key)).map(Long::parseLong).ifPresent(s -> result.put(type, s));
+            }
+            return result;
+        }
+
+        // CHECKSTYLE IGNORE: Method with many parameters
+        public static ClientConfig create(int port,
+                                          int maxRetries,
+                                          long millisToSleep,
+                                          long maxMillisToSleep,
+                                          long maxBufferSizeBytes,
+                                          long chunkSizeBytes,
+                                          int maxPoolSize,
+                                          int timeoutSeconds,
+                                          Map<FileType, Long> maxBufferOverride,
+                                          Map<FileType, Long> chunkBufferOverride)
+        {
+            return new ClientConfig(port,
+                                    maxRetries,
+                                    millisToSleep,
+                                    maxMillisToSleep,
+                                    maxBufferSizeBytes,
+                                    chunkSizeBytes,
+                                    maxPoolSize,
+                                    timeoutSeconds,
+                                    maxBufferOverride,
+                                    chunkBufferOverride);
+        }
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/clients/SidecarInstanceImpl.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/clients/SidecarInstanceImpl.java
new file mode 100644
index 0000000..2e58be6
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/clients/SidecarInstanceImpl.java
@@ -0,0 +1,122 @@
+/*
+ * 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.cassandra.clients;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.Serializable;
+import java.util.Objects;
+
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+import org.apache.cassandra.sidecar.client.SidecarInstance;
+
+/**
+ * A simple implementation of the {@link SidecarInstance} interface
+ */
+public class SidecarInstanceImpl implements Serializable, SidecarInstance
+{
+    private static final long serialVersionUID = -8650006905764842232L;
+    private static final Logger LOGGER = LoggerFactory.getLogger(SidecarInstanceImpl.class);
+
+    private int port;
+    private String hostname;
+
+    /**
+     * Constructs a new Sidecar instance with the given {@code port} and {@code hostname}
+     *
+     * @param hostname the host name where Sidecar is running
+     * @param port     the port where Sidecar is running
+     */
+    public SidecarInstanceImpl(String hostname, int port)
+    {
+        Preconditions.checkArgument(0 < port && port <= 65535,
+                                    "The Sidecar port number must be in the range 1-65535: %s", port);
+        this.port = port;
+        this.hostname = Objects.requireNonNull(hostname, "The Sidecar hostname must be non-null");
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public int port()
+    {
+        return port;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public String hostname()
+    {
+        return hostname;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public String toString()
+    {
+        return String.format("SidecarInstanceImpl{hostname='%s', port=%d}", hostname, port);
+    }
+
+    // JDK Serialization
+
+    private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException
+    {
+        LOGGER.warn("Falling back to JDK deserialization");
+        hostname = in.readUTF();
+        port = in.readInt();
+    }
+
+    private void writeObject(ObjectOutputStream out) throws IOException, ClassNotFoundException
+    {
+        LOGGER.warn("Falling back to JDK serialization");
+        out.writeUTF(hostname);
+        out.writeInt(port);
+    }
+
+    // Kryo Serialization
+
+    public static class Serializer extends com.esotericsoftware.kryo.Serializer<SidecarInstanceImpl>
+    {
+        @Override
+        public void write(Kryo kryo, Output out, SidecarInstanceImpl sidecarInstance)
+        {
+            out.writeString(sidecarInstance.hostname);
+            out.writeInt(sidecarInstance.port);
+        }
+
+        @Override
+        public SidecarInstanceImpl read(Kryo kryo, Input input, Class<SidecarInstanceImpl> type)
+        {
+            return new SidecarInstanceImpl(input.readString(), input.readInt());
+        }
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/clients/SidecarStreamConsumerAdapter.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/clients/SidecarStreamConsumerAdapter.java
new file mode 100644
index 0000000..0e48f0b
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/clients/SidecarStreamConsumerAdapter.java
@@ -0,0 +1,107 @@
+/*
+ * 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.cassandra.clients;
+
+import java.nio.ByteBuffer;
+import java.util.Objects;
+
+import org.apache.cassandra.sidecar.client.StreamBuffer;
+import org.apache.cassandra.spark.utils.streaming.StreamConsumer;
+
+/**
+ * A delegate class that connects Sidecar's {@link org.apache.cassandra.sidecar.client.StreamConsumer} to the analytics
+ * {@link StreamConsumer}
+ */
+public class SidecarStreamConsumerAdapter implements org.apache.cassandra.sidecar.client.StreamConsumer
+{
+    private final StreamConsumer delegate;
+
+    public SidecarStreamConsumerAdapter(StreamConsumer delegate)
+    {
+        this.delegate = delegate;
+    }
+
+    @Override
+    public void onRead(org.apache.cassandra.sidecar.client.StreamBuffer streamBuffer)
+    {
+        delegate.onRead(SidecarStreamBufferWrapper.wrap(streamBuffer));
+    }
+
+    @Override
+    public void onComplete()
+    {
+        delegate.onEnd();
+    }
+
+    @Override
+    public void onError(Throwable throwable)
+    {
+        delegate.onError(throwable);
+    }
+
+    /**
+     * A {@link org.apache.cassandra.spark.utils.streaming.StreamBuffer} implementations that internally wraps
+     * Sidecar's {@link org.apache.cassandra.sidecar.client.StreamBuffer}
+     */
+    static final class SidecarStreamBufferWrapper implements org.apache.cassandra.spark.utils.streaming.StreamBuffer
+    {
+        public final org.apache.cassandra.sidecar.client.StreamBuffer buffer;
+
+        private SidecarStreamBufferWrapper(StreamBuffer buffer)
+        {
+            this.buffer = Objects.requireNonNull(buffer, "the buffer parameter must be non-null");
+        }
+
+        public static SidecarStreamBufferWrapper wrap(StreamBuffer buffer)
+        {
+            return new SidecarStreamBufferWrapper(buffer);
+        }
+
+        @Override
+        public void getBytes(int index, ByteBuffer destination, int length)
+        {
+            buffer.copyBytes(index, destination, length);
+        }
+
+        @Override
+        public void getBytes(int index, byte[] destination, int destinationIndex, int length)
+        {
+            buffer.copyBytes(index, destination, destinationIndex, length);
+        }
+
+        @Override
+        public byte getByte(int index)
+        {
+            return buffer.getByte(index);
+        }
+
+        @Override
+        public int readableBytes()
+        {
+            return buffer.readableBytes();
+        }
+
+        @Override
+        public void release()
+        {
+            buffer.release();
+        }
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/clients/SslConfig.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/clients/SslConfig.java
new file mode 100644
index 0000000..2fd72b9
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/clients/SslConfig.java
@@ -0,0 +1,486 @@
+/*
+ * 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.cassandra.clients;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.Serializable;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.Base64;
+import java.util.Map;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+import org.apache.cassandra.spark.utils.MapUtils;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Encapsulates SSL configuration for Sidecar
+ */
+public class SslConfig implements Serializable
+{
+    private static final long serialVersionUID = -3844712192096436932L;
+    private static final Logger LOGGER = LoggerFactory.getLogger(SslConfig.class);
+    public static final String KEYSTORE_PATH = "KEYSTORE_PATH";
+    public static final String KEYSTORE_BASE64_ENCODED = "KEYSTORE_BASE64_ENCODED";
+    public static final String KEYSTORE_PASSWORD = "KEYSTORE_PASSWORD";
+    public static final String KEYSTORE_TYPE = "KEYSTORE_TYPE";
+    public static final String DEFAULT_KEYSTORE_TYPE = "PKCS12";
+    public static final String TRUSTSTORE_PATH = "TRUSTSTORE_PATH";
+    public static final String TRUSTSTORE_BASE64_ENCODED = "TRUSTSTORE_BASE64_ENCODED";
+    public static final String TRUSTSTORE_PASSWORD = "TRUSTSTORE_PASSWORD";
+    public static final String TRUSTSTORE_TYPE = "TRUSTSTORE_TYPE";
+    public static final String DEFAULT_TRUSTSTORE_TYPE = "JKS";
+    protected String keyStorePath;
+    protected String base64EncodedKeyStore;
+    protected String keyStorePassword;
+    protected String keyStoreType;
+    protected String trustStorePath;
+    protected String base64EncodedTrustStore;
+    protected String trustStorePassword;
+    protected String trustStoreType;
+
+    protected SslConfig(Builder<?> builder)
+    {
+        keyStorePath = builder.keyStorePath;
+        base64EncodedKeyStore = builder.base64EncodedKeyStore;
+        keyStorePassword = builder.keyStorePassword;
+        keyStoreType = builder.keyStoreType;
+        trustStorePath = builder.trustStorePath;
+        base64EncodedTrustStore = builder.base64EncodedTrustStore;
+        trustStorePassword = builder.trustStorePassword;
+        trustStoreType = builder.trustStoreType;
+    }
+
+    /**
+     * @return an input stream to the keystore source
+     * @throws IOException when an IO exception occurs
+     */
+    public InputStream keyStoreInputStream() throws IOException
+    {
+        if (keyStorePath != null)
+        {
+            return Files.newInputStream(Paths.get(keyStorePath));
+        }
+        else if (base64EncodedKeyStore != null)
+        {
+            return new ByteArrayInputStream(Base64.getDecoder().decode(base64EncodedKeyStore));
+        }
+        else
+        {
+            // It should never reach here
+            throw new RuntimeException("keyStorePath or encodedKeyStore must be provided");
+        }
+    }
+
+    /**
+     * @return the path to the key store file
+     */
+    public String keyStorePath()
+    {
+        return keyStorePath;
+    }
+
+    /**
+     * @return the encoded key store
+     */
+    public String base64EncodedKeyStore()
+    {
+        return base64EncodedKeyStore;
+    }
+
+    /**
+     * @return the key store password
+     */
+    public String keyStorePassword()
+    {
+        return keyStorePassword;
+    }
+
+    /**
+     * @return the key store type
+     */
+    public String keyStoreType()
+    {
+        return keyStoreType != null ? keyStoreType : DEFAULT_KEYSTORE_TYPE;
+    }
+
+    /**
+     * @return an input stream to the truststore source
+     * @throws IOException when an IO exception occurs
+     */
+    public InputStream trustStoreInputStream() throws IOException
+    {
+        if (trustStorePath != null)
+        {
+            return Files.newInputStream(Paths.get(trustStorePath));
+        }
+        else if (base64EncodedTrustStore != null)
+        {
+            return new ByteArrayInputStream(Base64.getDecoder().decode(base64EncodedTrustStore));
+        }
+        return null;
+    }
+
+    /**
+     * @return the path to the trust store
+     */
+    public String trustStorePath()
+    {
+        return trustStorePath;
+    }
+
+    /**
+     * @return the encoded trust store
+     */
+    public String base64EncodedTrustStore()
+    {
+        return base64EncodedTrustStore;
+    }
+
+    /**
+     * @return the trust store password
+     */
+    public String trustStorePassword()
+    {
+        return trustStorePassword;
+    }
+
+    /**
+     * @return the trust store type
+     */
+    public String trustStoreType()
+    {
+        return trustStoreType != null ? trustStoreType : DEFAULT_TRUSTSTORE_TYPE;
+    }
+
+    // JDK Serialization
+
+    private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException
+    {
+        LOGGER.warn("Falling back to JDK deserialization");
+        this.keyStorePath = readNullableString(in);
+        this.base64EncodedKeyStore = readNullableString(in);
+        this.keyStorePassword = readNullableString(in);
+        this.keyStoreType = readNullableString(in);
+        this.trustStorePath = readNullableString(in);
+        this.base64EncodedTrustStore = readNullableString(in);
+        this.trustStorePassword = readNullableString(in);
+        this.trustStoreType = readNullableString(in);
+    }
+
+    private void writeObject(ObjectOutputStream out) throws IOException
+    {
+        LOGGER.warn("Falling back to JDK serialization");
+        writeNullableString(out, keyStorePath);
+        writeNullableString(out, base64EncodedKeyStore);
+        writeNullableString(out, keyStorePassword);
+        writeNullableString(out, keyStoreType);
+        writeNullableString(out, trustStorePath);
+        writeNullableString(out, base64EncodedTrustStore);
+        writeNullableString(out, trustStorePassword);
+        writeNullableString(out, trustStoreType);
+    }
+
+    private String readNullableString(ObjectInputStream in) throws IOException
+    {
+        return in.readBoolean() ? in.readUTF() : null;
+    }
+
+    private void writeNullableString(ObjectOutputStream out, String string) throws IOException
+    {
+        if (string != null)
+        {
+            out.writeBoolean(true);
+            out.writeUTF(string);
+        }
+        else
+        {
+            out.writeBoolean(false);
+        }
+    }
+
+    // Kryo
+
+    public static class Serializer extends com.esotericsoftware.kryo.Serializer<SslConfig>
+    {
+        public SslConfig read(Kryo kryo, Input in, Class type)
+        {
+            return new Builder<>()
+                   .keyStorePath(in.readString())
+                   .base64EncodedKeyStore(in.readString())
+                   .keyStorePassword(in.readString())
+                   .keyStoreType(in.readString())
+                   .trustStorePath(in.readString())
+                   .base64EncodedTrustStore(in.readString())
+                   .trustStorePassword(in.readString())
+                   .trustStoreType(in.readString())
+                   .build();
+        }
+
+        public void write(Kryo kryo, Output out, SslConfig config)
+        {
+            out.writeString(config.keyStorePath);
+            out.writeString(config.base64EncodedKeyStore);
+            out.writeString(config.keyStorePassword);
+            out.writeString(config.keyStoreType);
+            out.writeString(config.trustStorePath);
+            out.writeString(config.base64EncodedTrustStore);
+            out.writeString(config.trustStorePassword);
+            out.writeString(config.trustStoreType);
+        }
+    }
+
+    @Nullable
+    public static SslConfig create(Map<String, String> options)
+    {
+        String keyStorePath = MapUtils.getOrDefault(options, KEYSTORE_PATH, null);
+        String encodedKeyStore = MapUtils.getOrDefault(options, KEYSTORE_BASE64_ENCODED, null);
+        String keyStorePassword = MapUtils.getOrDefault(options, KEYSTORE_PASSWORD, null);
+        String keyStoreType = MapUtils.getOrDefault(options, KEYSTORE_TYPE, null);
+        String trustStorePath = MapUtils.getOrDefault(options, TRUSTSTORE_PATH, null);
+        String encodedTrustStore = MapUtils.getOrDefault(options, TRUSTSTORE_BASE64_ENCODED, null);
+        String trustStorePassword = MapUtils.getOrDefault(options, TRUSTSTORE_PASSWORD, null);
+        String trustStoreType = MapUtils.getOrDefault(options, TRUSTSTORE_TYPE, null);
+
+        // If any of the values are provided we try to create a valid SecretsConfig object
+        if (keyStorePath != null
+                || encodedKeyStore != null
+                || keyStorePassword != null
+                || keyStoreType != null
+                || trustStorePath != null
+                || encodedTrustStore != null
+                || trustStorePassword != null
+                || trustStoreType != null)
+        {
+            Builder<?> validatedConfig = new Builder<>()
+                                         .keyStorePath(keyStorePath)
+                                         .base64EncodedKeyStore(encodedKeyStore)
+                                         .keyStorePassword(keyStorePassword)
+                                         .keyStoreType(keyStoreType)
+                                         .trustStorePath(trustStorePath)
+                                         .base64EncodedTrustStore(encodedTrustStore)
+                                         .trustStorePassword(trustStorePassword)
+                                         .trustStoreType(trustStoreType)
+                                         .validate();
+            LOGGER.info("Valid SSL configuration");
+            return validatedConfig.build();
+        }
+
+        LOGGER.warn("No SSL configured");
+        return null;
+    }
+
+    /**
+     * {@code SslConfig} builder static inner class
+     *
+     * @param <T> itself
+     */
+    public static class Builder<T extends SslConfig.Builder<T>>
+    {
+        protected String keyStorePath;
+        protected String base64EncodedKeyStore;
+        protected String keyStorePassword;
+        protected String keyStoreType;
+        protected String trustStorePath;
+        protected String base64EncodedTrustStore;
+        protected String trustStorePassword;
+        protected String trustStoreType;
+
+        /**
+         * @return a reference to itself
+         */
+        @SuppressWarnings("unchecked")
+        protected T self()
+        {
+            return (T) this;
+        }
+
+        /**
+         * Sets the {@code keyStorePath} and returns a reference to this Builder enabling method chaining
+         *
+         * @param keyStorePath the {@code keyStorePath} to set
+         * @return a reference to this Builder
+         */
+        public T keyStorePath(String keyStorePath)
+        {
+            this.keyStorePath = keyStorePath;
+            return self();
+        }
+
+        /**
+         * Sets the {@code base64EncodedKeyStore} and returns a reference to this Builder enabling method chaining
+         *
+         * @param base64EncodedKeyStore the {@code base64EncodedKeyStore} to set
+         * @return a reference to this Builder
+         */
+        public T base64EncodedKeyStore(String base64EncodedKeyStore)
+        {
+            this.base64EncodedKeyStore = base64EncodedKeyStore;
+            return self();
+        }
+
+        /**
+         * Sets the {@code keyStorePassword} and returns a reference to this Builder enabling method chaining
+         *
+         * @param keyStorePassword the {@code keyStorePassword} to set
+         * @return a reference to this Builder
+         */
+        public T keyStorePassword(String keyStorePassword)
+        {
+            this.keyStorePassword = keyStorePassword;
+            return self();
+        }
+
+        /**
+         * Sets the {@code keyStoreType} and returns a reference to this Builder enabling method chaining
+         *
+         * @param keyStoreType the {@code keyStoreType} to set
+         * @return a reference to this Builder
+         */
+        public T keyStoreType(String keyStoreType)
+        {
+            this.keyStoreType = keyStoreType;
+            return self();
+        }
+
+        /**
+         * Sets the {@code trustStorePath} and returns a reference to this Builder enabling method chaining
+         *
+         * @param trustStorePath the {@code trustStorePath} to set
+         * @return a reference to this Builder
+         */
+        public T trustStorePath(String trustStorePath)
+        {
+            this.trustStorePath = trustStorePath;
+            return self();
+        }
+
+        /**
+         * Sets the {@code base64EncodedTrustStore} and returns a reference to this Builder enabling method chaining
+         *
+         * @param base64EncodedTrustStore the {@code base64EncodedTrustStore} to set
+         * @return a reference to this Builder
+         */
+        public T base64EncodedTrustStore(String base64EncodedTrustStore)
+        {
+            this.base64EncodedTrustStore = base64EncodedTrustStore;
+            return self();
+        }
+
+        /**
+         * Sets the {@code trustStorePassword} and returns a reference to this Builder enabling method chaining
+         *
+         * @param trustStorePassword the {@code trustStorePassword} to set
+         * @return a reference to this Builder
+         */
+        public T trustStorePassword(String trustStorePassword)
+        {
+            this.trustStorePassword = trustStorePassword;
+            return self();
+        }
+
+        /**
+         * Sets the {@code trustStoreType} and returns a reference to this Builder enabling method chaining
+         *
+         * @param trustStoreType the {@code trustStoreType} to set
+         * @return a reference to this Builder
+         */
+        public T trustStoreType(String trustStoreType)
+        {
+            this.trustStoreType = trustStoreType;
+            return self();
+        }
+
+        /**
+         * Returns a {@code SslConfig} built from the parameters previously set
+         *
+         * @return a {@code SslConfig} built with parameters of this {@code SslConfig.Builder}
+         */
+        public SslConfig build()
+        {
+            return new SslConfig(this);
+        }
+
+        protected T validate()
+        {
+            if (keyStorePath != null && base64EncodedKeyStore != null)
+            {
+                throw new IllegalArgumentException(
+                        String.format("Both '%s' and '%s' options were provided. Only one of the options can be provided",
+                                      KEYSTORE_PATH, KEYSTORE_BASE64_ENCODED));
+            }
+
+            if (keyStorePassword != null)
+            {
+                // When the keystore password is provided, either the key store path or
+                // the encoded key store must be provided
+                if (keyStorePath == null && base64EncodedKeyStore == null)
+                {
+                    throw new IllegalArgumentException(
+                            String.format("One of the '%s' or '%s' options must be provided when the '%s' option is provided",
+                                          KEYSTORE_PATH, KEYSTORE_BASE64_ENCODED, KEYSTORE_PASSWORD));
+                }
+            }
+            else
+            {
+
+                throw new IllegalArgumentException(
+                        String.format("The '%s' option must be provided when either the '%s' or '%s' options are provided",
+                                      KEYSTORE_PASSWORD, KEYSTORE_PATH, KEYSTORE_BASE64_ENCODED));
+            }
+
+            if (trustStorePassword != null)
+            {
+                // Only one of trust store path or encoded trust store must be provided, not both
+                if (trustStorePath != null && base64EncodedTrustStore != null)
+                {
+                    throw new IllegalArgumentException(
+                            String.format("Both '%s' and '%s' options were provided. Only one of the options can be provided",
+                                          TRUSTSTORE_PATH, TRUSTSTORE_BASE64_ENCODED));
+                }
+
+                // When the trust store password is provided, either the trust store
+                // path or the encoded trust store must be provided
+                if (trustStorePath == null && base64EncodedTrustStore == null)
+                {
+                    throw new IllegalArgumentException(
+                            String.format("One of the '%s' or '%s' options must be provided when the '%s' option is provided",
+                                          TRUSTSTORE_PATH, TRUSTSTORE_BASE64_ENCODED, TRUSTSTORE_PASSWORD));
+                }
+            }
+            else if (trustStorePath != null || base64EncodedTrustStore != null)
+            {
+                throw new IllegalArgumentException(
+                        String.format("The '%s' option must be provided when either the '%s' or '%s' options are provided",
+                                      TRUSTSTORE_PASSWORD, TRUSTSTORE_PATH, TRUSTSTORE_BASE64_ENCODED));
+            }
+            return self();
+        }
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/KryoRegister.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/KryoRegister.java
new file mode 100644
index 0000000..d00b705
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/KryoRegister.java
@@ -0,0 +1,108 @@
+/*
+ * 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.cassandra.spark;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.Serializer;
+import org.apache.cassandra.bridge.BigNumberConfigImpl;
+import org.apache.cassandra.bridge.CassandraBridgeFactory;
+import org.apache.cassandra.bridge.CassandraVersion;
+import org.apache.cassandra.clients.SidecarInstanceImpl;
+import org.apache.cassandra.clients.SslConfig;
+import org.apache.cassandra.spark.data.CassandraDataLayer;
+import org.apache.cassandra.spark.data.LocalDataLayer;
+import org.apache.cassandra.spark.data.ReplicationFactor;
+import org.apache.cassandra.spark.data.partitioner.CassandraInstance;
+import org.apache.cassandra.spark.data.partitioner.CassandraRing;
+import org.apache.cassandra.spark.data.partitioner.TokenPartitioner;
+import org.apache.spark.SparkConf;
+import org.apache.spark.serializer.KryoRegistrator;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Helper class to register classes for Kryo serialization
+ */
+public class KryoRegister implements KryoRegistrator
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(KryoRegister.class);
+    private static final String SPARK_SERIALIZER = "spark.serializer";
+    private static final String SPARK_REGISTRATORS = "spark.kryo.registrator";
+    private static final Map<Class<?>, Serializer<?>> KRYO_SERIALIZERS = Collections.synchronizedMap(new LinkedHashMap<>());
+
+    static
+    {
+        // Cassandra-version-agnostic Kryo serializers
+        KRYO_SERIALIZERS.put(LocalDataLayer.class, new LocalDataLayer.Serializer());
+        KRYO_SERIALIZERS.put(CassandraInstance.class, new CassandraInstance.Serializer());
+        KRYO_SERIALIZERS.put(ReplicationFactor.class, new ReplicationFactor.Serializer());
+        KRYO_SERIALIZERS.put(CassandraRing.class, new CassandraRing.Serializer());
+        KRYO_SERIALIZERS.put(TokenPartitioner.class, new TokenPartitioner.Serializer());
+        KRYO_SERIALIZERS.put(CassandraDataLayer.class, new CassandraDataLayer.Serializer());
+        KRYO_SERIALIZERS.put(BigNumberConfigImpl.class, new BigNumberConfigImpl.Serializer());
+        KRYO_SERIALIZERS.put(SidecarInstanceImpl.class, new SidecarInstanceImpl.Serializer());
+        KRYO_SERIALIZERS.put(SslConfig.class, new SslConfig.Serializer());
+    }
+
+    public static <T> void addSerializer(@NotNull Class<T> type, @NotNull Serializer<T> serializer)
+    {
+        LOGGER.info("Registering custom Kryo serializer type={}", type.getName());
+        KRYO_SERIALIZERS.put(type, serializer);
+    }
+
+    @Override
+    public void registerClasses(@NotNull Kryo kryo)
+    {
+        LOGGER.info("Initializing KryoRegister");
+
+        // TODO: Implicitly defaulting to Cassandra version 4.0 is a part of a previously published API.
+        //       We might want to persist the version of Cassandra into the Spark configuration instead.
+        CassandraBridgeFactory.get(CassandraVersion.FOURZERO).kryoRegister(kryo);
+
+        KRYO_SERIALIZERS.forEach(kryo::register);
+    }
+
+    public static void setup(@NotNull SparkConf configuration)
+    {
+        // Use KryoSerializer
+        LOGGER.info("Setting up Kryo");
+        configuration.set(SPARK_SERIALIZER, "org.apache.spark.serializer.KryoSerializer");
+
+        // Add KryoRegister to SparkConf serialization if not already there
+        Set<String> registratorsSet = Arrays.stream(configuration.get(SPARK_REGISTRATORS, "").split(","))
+                                            .filter(string -> string != null && !string.isEmpty())
+                                            .collect(Collectors.toSet());
+        registratorsSet.add(KryoRegister.class.getName());
+        String registratorsString = String.join(",", registratorsSet);
+        LOGGER.info("Setting kryo registrators: " + registratorsString);
+        configuration.set(SPARK_REGISTRATORS, registratorsString);
+
+        configuration.registerKryoClasses(new Class<?>[]{KryoRegister.class});
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/BulkSparkConf.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/BulkSparkConf.java
new file mode 100644
index 0000000..292282f
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/BulkSparkConf.java
@@ -0,0 +1,464 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+import java.io.ByteArrayInputStream;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.InputStream;
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Base64;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.bridge.CassandraBridgeFactory;
+import org.apache.cassandra.clients.SidecarInstanceImpl;
+import org.apache.cassandra.sidecar.client.SidecarInstance;
+import org.apache.cassandra.spark.bulkwriter.token.ConsistencyLevel;
+import org.apache.cassandra.spark.bulkwriter.util.SbwKryoRegistrator;
+import org.apache.cassandra.spark.utils.BuildInfo;
+import org.apache.cassandra.spark.utils.MapUtils;
+import org.apache.spark.SparkConf;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+@SuppressWarnings("WeakerAccess")
+public class BulkSparkConf implements Serializable
+{
+    private static final long serialVersionUID = -5060973521517656241L;
+    private static final Logger LOGGER = LoggerFactory.getLogger(BulkSparkConf.class);
+
+    public static final String JDK11_OPTIONS = " -Djdk.attach.allowAttachSelf=true"
+                                             + " --add-exports java.base/jdk.internal.misc=ALL-UNNAMED"
+                                             + " --add-exports java.base/jdk.internal.ref=ALL-UNNAMED"
+                                             + " --add-exports java.base/sun.nio.ch=ALL-UNNAMED"
+                                             + " --add-exports java.management.rmi/com.sun.jmx.remote.internal.rmi=ALL-UNNAMED"
+                                             + " --add-exports java.rmi/sun.rmi.registry=ALL-UNNAMED"
+                                             + " --add-exports java.rmi/sun.rmi.server=ALL-UNNAMED"
+                                             + " --add-exports java.sql/java.sql=ALL-UNNAMED"
+                                             + " --add-opens java.base/java.lang.module=ALL-UNNAMED"
+                                             + " --add-opens java.base/jdk.internal.loader=ALL-UNNAMED"
+                                             + " --add-opens java.base/jdk.internal.ref=ALL-UNNAMED"
+                                             + " --add-opens java.base/jdk.internal.reflect=ALL-UNNAMED"
+                                             + " --add-opens java.base/jdk.internal.math=ALL-UNNAMED"
+                                             + " --add-opens java.base/jdk.internal.module=ALL-UNNAMED"
+                                             + " --add-opens java.base/jdk.internal.util.jar=ALL-UNNAMED"
+                                             + " --add-opens jdk.management/com.sun.management.internal=ALL-UNNAMED";
+
+    public static final int DEFAULT_NUM_SPLITS = -1;
+    public static final int DEFAULT_HTTP_CONNECTION_TIMEOUT = 100_000;
+    public static final int DEFAULT_HTTP_RESPONSE_TIMEOUT = 100_000;
+    public static final int DEFAULT_HTTP_MAX_CONNECTIONS = 200;
+    public static final int DEFAULT_SIDECAR_PORT = 9043;
+    public static final int DEFAULT_SIDECAR_REQUEST_RETRIES = 10;
+    public static final long DEFAULT_SIDECAR_REQUEST_RETRY_DELAY_SECONDS = 1L;
+    public static final long DEFAULT_SIDECAR_REQUEST_MAX_RETRY_DELAY_SECONDS = 60L;
+    public static final int DEFAULT_COMMIT_BATCH_SIZE = 10_000;
+    public static final int DEFAULT_RING_RETRY_COUNT = 3;
+    public static final RowBufferMode DEFAULT_ROW_BUFFER_MODE = RowBufferMode.UNBUFFERRED;
+    public static final int DEFAULT_BATCH_SIZE_IN_ROWS = 1_000_000;
+
+    // NOTE: All Cassandra Analytics setting names must start with "spark" in order to not be ignored by Spark,
+    //       and must not start with "spark.cassandra" so as to not conflict with Spark Cassandra Connector
+    //       which will throw a configuration exception for each setting with that prefix it does not recognize
+    public static final String SETTING_PREFIX = "spark.cassandra_analytics.";
+
+    public static final String HTTP_MAX_CONNECTIONS                    = SETTING_PREFIX + "request.max_connections";
+    public static final String HTTP_RESPONSE_TIMEOUT                   = SETTING_PREFIX + "request.response_timeout";
+    public static final String HTTP_CONNECTION_TIMEOUT                 = SETTING_PREFIX + "request.connection_timeout";
+    public static final String SIDECAR_PORT                            = SETTING_PREFIX + "ports.sidecar";
+    public static final String SIDECAR_REQUEST_RETRIES                 = SETTING_PREFIX + "sidecar.request.retries";
+    public static final String SIDECAR_REQUEST_RETRY_DELAY_SECONDS     = SETTING_PREFIX + "sidecar.request.retries.delay.seconds";
+    public static final String SIDECAR_REQUEST_MAX_RETRY_DELAY_SECONDS = SETTING_PREFIX + "sidecar.request.retries.max.delay.seconds";
+    public static final String SKIP_CLEAN                              = SETTING_PREFIX + "job.skip_clean";
+    public static final String USE_OPENSSL                             = SETTING_PREFIX + "use_openssl";
+    public static final String RING_RETRY_COUNT                        = SETTING_PREFIX + "ring_retry_count";
+
+    public final Set<? extends SidecarInstance> sidecarInstances;
+    public final String keyspace;
+    public final String table;
+    public final ConsistencyLevel.CL consistencyLevel;
+    public final String localDC;
+    public final Integer numberSplits;
+    public final RowBufferMode rowBufferMode;
+    public final Integer sstableDataSizeInMB;
+    public final Integer sstableBatchSize;
+    public final int commitBatchSize;
+    public final boolean skipExtendedVerify;
+    public final WriteMode writeMode;
+    protected final String keystorePassword;
+    protected final String keystorePath;
+    protected final String keystoreBase64Encoded;
+    protected final String keystoreType;
+    protected final String truststorePassword;
+    protected final String truststorePath;
+    protected final String truststoreBase64Encoded;
+    protected final String truststoreType;
+    protected final SparkConf conf;
+    public final boolean validateSSTables;
+    public final int commitThreadsPerInstance;
+    protected final int sidecarPort;
+    protected boolean useOpenSsl;
+    protected int ringRetryCount;
+
+    public BulkSparkConf(SparkConf conf, Map<String, String> options)
+    {
+        this.conf = conf;
+        this.sidecarPort = MapUtils.getInt(options, WriterOptions.SIDECAR_PORT.name(), getInt(SIDECAR_PORT, DEFAULT_SIDECAR_PORT), "sidecar port");
+        this.sidecarInstances = buildSidecarInstances(options, sidecarPort);
+        this.keyspace = MapUtils.getOrThrow(options, WriterOptions.KEYSPACE.name());
+        this.table = MapUtils.getOrThrow(options, WriterOptions.TABLE.name());
+        this.validateSSTables = MapUtils.getBoolean(options, WriterOptions.VALIDATE_SSTABLES.name(), true, "validate SSTables");
+        this.skipExtendedVerify = MapUtils.getBoolean(options, WriterOptions.SKIP_EXTENDED_VERIFY.name(), true,
+                                                      "skip extended verification of SSTables by Cassandra");
+        this.consistencyLevel = ConsistencyLevel.CL.valueOf(MapUtils.getOrDefault(options, WriterOptions.BULK_WRITER_CL.name(), "EACH_QUORUM"));
+        this.localDC = MapUtils.getOrDefault(options, WriterOptions.LOCAL_DC.name(), null);
+        this.numberSplits = MapUtils.getInt(options, WriterOptions.NUMBER_SPLITS.name(), DEFAULT_NUM_SPLITS, "number of splits");
+        this.rowBufferMode = MapUtils.getEnumOption(options, WriterOptions.ROW_BUFFER_MODE.name(), DEFAULT_ROW_BUFFER_MODE, "row bufferring mode");
+        this.sstableDataSizeInMB = MapUtils.getInt(options, WriterOptions.SSTABLE_DATA_SIZE_IN_MB.name(), 160, "sstable data size in MB");
+        this.sstableBatchSize = MapUtils.getInt(options, WriterOptions.BATCH_SIZE.name(), 1_000_000, "sstable batch size");
+        this.commitBatchSize = MapUtils.getInt(options, WriterOptions.COMMIT_BATCH_SIZE.name(), DEFAULT_COMMIT_BATCH_SIZE, "commit batch size");
+        this.commitThreadsPerInstance = MapUtils.getInt(options, WriterOptions.COMMIT_THREADS_PER_INSTANCE.name(), 2, "commit threads per instance");
+        this.keystorePassword = MapUtils.getOrDefault(options, WriterOptions.KEYSTORE_PASSWORD.name(), null);
+        this.keystorePath = MapUtils.getOrDefault(options, WriterOptions.KEYSTORE_PATH.name(), null);
+        this.keystoreBase64Encoded = MapUtils.getOrDefault(options, WriterOptions.KEYSTORE_BASE64_ENCODED.name(), null);
+        this.keystoreType = MapUtils.getOrDefault(options, WriterOptions.KEYSTORE_TYPE.name(), "PKCS12");
+        this.truststorePassword = MapUtils.getOrDefault(options, WriterOptions.TRUSTSTORE_PASSWORD.name(), null);
+        this.truststorePath = MapUtils.getOrDefault(options, WriterOptions.TRUSTSTORE_PATH.name(), null);
+        this.truststoreBase64Encoded = MapUtils.getOrDefault(options, WriterOptions.TRUSTSTORE_BASE64_ENCODED.name(), null);
+        this.truststoreType = MapUtils.getOrDefault(options, WriterOptions.TRUSTSTORE_TYPE.name(), null);
+        this.writeMode = MapUtils.getEnumOption(options, WriterOptions.WRITE_MODE.name(), WriteMode.INSERT, "write mode");
+        // For backwards-compatibility with port settings, use writer option if available,
+        // else fall back to props, and then default if neither specified
+        this.useOpenSsl = getBoolean(USE_OPENSSL, true);
+        this.ringRetryCount = getInt(RING_RETRY_COUNT, DEFAULT_RING_RETRY_COUNT);
+        validateEnvironment();
+    }
+
+    protected Set<? extends SidecarInstance> buildSidecarInstances(Map<String, String> options, int sidecarPort)
+    {
+        String sidecarInstances = MapUtils.getOrThrow(options, WriterOptions.SIDECAR_INSTANCES.name(), "sidecar_instances");
+        return Arrays.stream(sidecarInstances.split(","))
+                     .map(hostname -> new SidecarInstanceImpl(hostname, sidecarPort))
+                     .collect(Collectors.toSet());
+    }
+
+    protected void validateEnvironment() throws RuntimeException
+    {
+        Preconditions.checkNotNull(keyspace);
+        Preconditions.checkNotNull(table);
+        Preconditions.checkArgument(getHttpResponseTimeoutMs() > 0, HTTP_RESPONSE_TIMEOUT + " must be > 0");
+        validateSslConfiguration();
+        validateTableWriterSettings();
+        CassandraBridgeFactory.validateBridges();
+    }
+
+    protected void validateTableWriterSettings()
+    {
+        boolean batchSizeIsZero = sstableBatchSize == 0;
+        if (rowBufferMode == RowBufferMode.BUFFERRED
+                && (!batchSizeIsZero && sstableBatchSize != DEFAULT_BATCH_SIZE_IN_ROWS))
+        {
+            LOGGER.warn("BATCH_SIZE is set to a non-zero, non-default value ({}) but ROW_BUFFER_MODE is set to BUFFERRED."
+                      + " Ignoring BATCH_SIZE.", sstableBatchSize);
+        }
+        Preconditions.checkArgument(rowBufferMode == RowBufferMode.UNBUFFERRED && !batchSizeIsZero,
+                                    "If writing in sorted order (ROW_BUFFER_MODE is UNBUFFERRED) then BATCH_SIZE "
+                                  + "should be non zero, but it was set to 0 in writer options");
+    }
+
+    /*
+     * This method Will throw if the SSL configuration is incorrect (PATH provided w/o password, for example)
+     */
+    protected void validateSslConfiguration()
+    {
+        if (getKeyStorePassword() != null)
+        {
+            // If the mTLS keystore password is provided, we validate that the path or a base64 keystore is provided
+            if (getKeyStorePath() == null && getKeystoreBase64Encoded() == null)
+            {
+                throw new NullPointerException("Keystore password was set. "
+                                             + "But both keystore path and base64 encoded string are not set. "
+                                             + "Please either set option " + WriterOptions.KEYSTORE_PATH
+                                             + " or option " + WriterOptions.KEYSTORE_BASE64_ENCODED);
+            }
+        }
+
+        // Check to make sure if either trust store password or trust store path are specified, both are specified
+        if (getConfiguredTrustStorePassword() != null)
+        {
+            Preconditions.checkArgument(getTruststoreBase64Encoded() != null || getTrustStorePath() != null,
+                                        "Trust Store password was provided, but both truststore path and base64 encoded string are missing. "
+                                      + "Please provide either option " + WriterOptions.TRUSTSTORE_PATH
+                                      + " or option " + WriterOptions.TRUSTSTORE_BASE64_ENCODED);
+        }
+        else
+        {
+            if (getTrustStorePath() != null || getTruststoreBase64Encoded() != null)
+            {
+                Preconditions.checkNotNull(getTruststoreBase64Encoded(),
+                                           "Trust Store Path was provided, but password is missing. "
+                                         + "Please provide option " + WriterOptions.TRUSTSTORE_PASSWORD);
+                Preconditions.checkNotNull(getTrustStorePath(),
+                                           "Trust Store Base64 encoded was provided, but password is missing."
+                                         + "Please provide option " + WriterOptions.TRUSTSTORE_PASSWORD);
+            }
+        }
+    }
+
+    public int getSidecarPort()
+    {
+        return sidecarPort;
+    }
+
+    protected String getTrustStorePath()
+    {
+        return truststorePath;
+    }
+
+    protected String getTruststoreBase64Encoded()
+    {
+        return truststoreBase64Encoded;
+    }
+
+    public String getTrustStoreTypeOrDefault()
+    {
+        return truststoreType != null ? truststoreType : "PKCS12";
+    }
+
+    protected String getKeyStorePath()
+    {
+        return keystorePath;
+    }
+
+    protected String getKeystoreBase64Encoded()
+    {
+        return keystoreBase64Encoded;
+    }
+
+    public InputStream getKeyStore()
+    {
+        return getKeyStorePath() != null
+               ? getKeyStoreFromPath(getKeyStorePath())
+               : getKeyStoreFromBase64EncodedString(getKeystoreBase64Encoded());
+    }
+
+    @Nullable
+    public InputStream getTrustStore()
+    {
+        return getTrustStorePath() != null
+               ? getKeyStoreFromPath(getTrustStorePath())
+               : getKeyStoreFromBase64EncodedString(getTruststoreBase64Encoded());
+    }
+
+    protected InputStream getKeyStoreFromPath(String keyStorePath)
+    {
+        if (keyStorePath != null)
+        {
+            try
+            {
+                return new FileInputStream(keyStorePath);
+            }
+            catch (FileNotFoundException exception)
+            {
+                throw new RuntimeException("Could not load keystore at path '" + keyStorePath + "'", exception);
+            }
+        }
+        return null;
+    }
+
+    protected InputStream getKeyStoreFromBase64EncodedString(String keyStoreBase64Encoded)
+    {
+        if (keyStoreBase64Encoded != null)
+        {
+            return new ByteArrayInputStream(Base64.getDecoder().decode(keyStoreBase64Encoded));
+        }
+        return null;
+    }
+
+    public String getTrustStorePasswordOrDefault()
+    {
+        return truststorePassword != null ? truststorePassword : "password";
+    }
+
+    public String getKeyStoreTypeOrDefault()
+    {
+        return keystoreType != null ? keystoreType : "PKCS12";
+    }
+
+    public String getKeyStorePassword()
+    {
+        return keystorePassword;
+    }
+
+    public String getConfiguredTrustStorePassword()
+    {
+        return truststorePassword;
+    }
+
+    public String getConfiguredKeyStorePassword()
+    {
+        return keystorePassword;
+    }
+
+    public int getSidecarRequestRetries()
+    {
+        return getInt(SIDECAR_REQUEST_RETRIES, DEFAULT_SIDECAR_REQUEST_RETRIES);
+    }
+
+    public long getSidecarRequestRetryDelayInSeconds()
+    {
+        return getLong(SIDECAR_REQUEST_RETRY_DELAY_SECONDS, DEFAULT_SIDECAR_REQUEST_RETRY_DELAY_SECONDS);
+    }
+
+    public long getSidecarRequestMaxRetryDelayInSeconds()
+    {
+        return getLong(SIDECAR_REQUEST_MAX_RETRY_DELAY_SECONDS, DEFAULT_SIDECAR_REQUEST_MAX_RETRY_DELAY_SECONDS);
+    }
+
+    public int getHttpConnectionTimeoutMs()
+    {
+        return getInt(HTTP_CONNECTION_TIMEOUT, DEFAULT_HTTP_CONNECTION_TIMEOUT);
+    }
+
+    public int getHttpResponseTimeoutMs()
+    {
+        return getInt(HTTP_RESPONSE_TIMEOUT, DEFAULT_HTTP_RESPONSE_TIMEOUT);
+    }
+
+    public int getMaxHttpConnections()
+    {
+        return getInt(HTTP_MAX_CONNECTIONS, DEFAULT_HTTP_MAX_CONNECTIONS);
+    }
+
+    public boolean getSkipClean()
+    {
+        return getBoolean(SKIP_CLEAN, false);
+    }
+
+    public Integer getCores()
+    {
+        int coresPerExecutor = conf.getInt("spark.executor.cores", 1);
+        int numExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", conf.getInt("spark.executor.instances", 1));
+        return coresPerExecutor * numExecutors;
+    }
+
+    protected int getInt(String settingName, int defaultValue)
+    {
+        String finalSetting = getSettingNameOrDeprecatedName(settingName);
+        return conf.getInt(finalSetting, defaultValue);
+    }
+
+    protected long getLong(String settingName, long defaultValue)
+    {
+        String finalSetting = getSettingNameOrDeprecatedName(settingName);
+        return conf.getLong(finalSetting, defaultValue);
+    }
+
+    protected boolean getBoolean(String settingName, boolean defaultValue)
+    {
+        String finalSetting = getSettingNameOrDeprecatedName(settingName);
+        return conf.getBoolean(finalSetting, defaultValue);
+    }
+
+    protected String getSettingNameOrDeprecatedName(String settingName)
+    {
+        if (!conf.contains(settingName))
+        {
+            String settingSuffix = settingName.startsWith(SETTING_PREFIX)
+                    ? settingName.substring(SETTING_PREFIX.length())
+                    : settingName;
+            for (String settingPrefix : getDeprecatedSettingPrefixes())
+            {
+                String deprecatedSetting = settingPrefix + settingSuffix;
+                if (conf.contains(deprecatedSetting))
+                {
+                    LOGGER.warn("Found deprecated setting '{}'. Please use {} in the future.",
+                                deprecatedSetting, settingName);
+                    return deprecatedSetting;
+                }
+            }
+        }
+        return settingName;
+    }
+
+    @NotNull
+    protected List<String> getDeprecatedSettingPrefixes()
+    {
+        return Collections.emptyList();
+    }
+
+    /**
+     * The SBW utilizes Cassandra libraries to generate SSTables. Under JDK11, this library needs additional JVM options
+     * to be set for the executors for some backward-compatibility reasons. This method will add the appropriate
+     * JVM options. Additionally, we set up the SBW KryoRegistrator here rather than requiring the end-user
+     * to call 2 static methods to set up the appropriate settings for the job.
+     *
+     * @param conf               the Spark Configuration to set up
+     * @param addKryoRegistrator passs true if your application hasn't separately added
+     *                           the bulk-specific Kryo registrator, false if you have set it up
+     *                           separately (see the usage docs for more details)
+     */
+    public static void setupSparkConf(SparkConf conf, boolean addKryoRegistrator)
+    {
+        String previousOptions = conf.get("spark.executor.extraJavaOptions", "");
+        if (BuildInfo.isAtLeastJava11(BuildInfo.javaSpecificationVersion()))
+        {
+            conf.set("spark.executor.extraJavaOptions", previousOptions + JDK11_OPTIONS);
+        }
+
+        if (addKryoRegistrator)
+        {
+            // Use `SbwKryoRegistrator.setupKryoRegistrator(conf);` to add the Spark Bulk Writer's custom
+            // KryoRegistrator. This needs to happen before the SparkSession is built in order for it to work properly.
+            // This utility method will add the necessary configuration, but it may be overwritten later if your own
+            // code resets `spark.kryo.registrator` after this is called.
+            SbwKryoRegistrator.setupKryoRegistrator(conf);
+        }
+    }
+
+    protected SparkConf getConf()
+    {
+        return conf;
+    }
+
+    public boolean getUseOpenSsl()
+    {
+        return useOpenSsl;
+    }
+
+    public int getRingRetryCount()
+    {
+        return ringRetryCount;
+    }
+
+    public boolean hasKeystoreAndKeystorePassword()
+    {
+        return keystorePassword != null && (keystorePath != null || keystoreBase64Encoded != null);
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/BulkWriteValidator.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/BulkWriteValidator.java
new file mode 100644
index 0000000..1531738
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/BulkWriteValidator.java
@@ -0,0 +1,169 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+import java.math.BigInteger;
+import java.util.AbstractMap;
+import java.util.Collection;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+
+import com.google.common.collect.Multimap;
+import com.google.common.collect.Range;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.spark.bulkwriter.token.ReplicaAwareFailureHandler;
+
+public class BulkWriteValidator implements AutoCloseable
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(BulkWriteValidator.class);
+
+    private final ReplicaAwareFailureHandler<RingInstance> failureHandler;
+    private final CassandraRingMonitor monitor;
+    private final JobInfo job;
+    private String phase = "Initializing";
+
+    private final ClusterInfo cluster;
+
+    public BulkWriteValidator(BulkWriterContext bulkWriterContext,
+                              Consumer<CancelJobEvent> cancelJobFunc) throws Exception
+    {
+        cluster = bulkWriterContext.cluster();
+        job = bulkWriterContext.job();
+        failureHandler = new ReplicaAwareFailureHandler<>(cluster.getRing(true));
+        monitor = new CassandraRingMonitor(cluster, cancelJobFunc, 1000, TimeUnit.MILLISECONDS);
+    }
+
+    public void setPhase(String phase)
+    {
+        this.phase = phase;
+    }
+
+    public String getPhase()
+    {
+        return phase;
+    }
+
+    public void validateInitialEnvironment()
+    {
+        validateCLOrFail();
+    }
+
+    public void close()
+    {
+        monitor.stop();
+    }
+
+    private void validateCLOrFail()
+    {
+        updateInstanceAvailability();
+        validateClOrFail(failureHandler, LOGGER, phase, job);
+    }
+
+    public static void validateClOrFail(ReplicaAwareFailureHandler<RingInstance> failureHandler,
+                                        Logger logger,
+                                        String phase,
+                                        JobInfo job)
+    {
+        Collection<AbstractMap.SimpleEntry<Range<BigInteger>, Multimap<RingInstance, String>>> failedRanges =
+                failureHandler.getFailedEntries(job.getConsistencyLevel(), job.getLocalDC());
+        if (failedRanges.isEmpty())
+        {
+            logger.info("Succeeded {} with {}", phase, job.getConsistencyLevel());
+        }
+        else
+        {
+            String message = String.format("Failed to load %s ranges with %s for job %s in phase %s",
+                                           failedRanges.size(), job.getConsistencyLevel(), job.getId(), phase);
+            logger.error(message);
+            failedRanges.forEach(failedRange -> failedRange.getValue().keySet().forEach(instance ->
+                    logger.error("Failed {} for {} on {}", phase, failedRange.getKey(), instance.toString())));
+            throw new RuntimeException(message);
+        }
+    }
+
+    public static void updateFailureHandler(CommitResult commitResult,
+                                            String phase,
+                                            ReplicaAwareFailureHandler<RingInstance> failureHandler)
+    {
+        LOGGER.debug("Commit Result: {}", commitResult);
+        commitResult.failures.forEach((uuid, err) -> {
+            LOGGER.warn("[{}]: {} failed on {} with message {}",
+                    uuid, phase, commitResult.instance.getNodeName(), err.errMsg);
+            failureHandler.addFailure(err.tokenRange, commitResult.instance, err.errMsg);
+        });
+    }
+
+    private void updateInstanceAvailability()
+    {
+        cluster.refreshClusterInfo();
+        Map<RingInstance, InstanceAvailability> availability = cluster.getInstanceAvailability();
+        availability.forEach(this::checkInstance);
+    }
+
+    private void checkInstance(RingInstance instance, InstanceAvailability availability)
+    {
+        throwIfInvalidState(instance, availability);
+        updateFailureHandler(instance, availability);
+    }
+
+    private void throwIfInvalidState(RingInstance instance, InstanceAvailability availability)
+    {
+        if (availability == InstanceAvailability.INVALID_STATE)
+        {
+            // If we find any nodes in a totally invalid state, just throw as we can't continue
+            String message = String.format("Instance (%s) is in an invalid state (%s) during import. "
+                                         + "Please rerun import once topology changes are complete.",
+                                           instance.getNodeName(), cluster.getInstanceState(instance));
+            throw new RuntimeException(message);
+        }
+    }
+
+    private void updateFailureHandler(RingInstance instX, InstanceAvailability availability)
+    {
+        if (availability != InstanceAvailability.AVAILABLE)
+        {
+            addFailedInstance(instX, availability.getMessage());
+        }
+    }
+
+    private void addFailedInstance(RingInstance instance, String reason)
+    {
+        Collection<Range<BigInteger>> failedRanges = cluster.getRing(true).getTokenRanges(instance);
+        failedRanges.forEach(failedRange -> {
+            String nodeDisplayName = instance.getNodeName();
+            String message = String.format("%s %s", nodeDisplayName, reason);
+            LOGGER.warn("{} failed in phase {} on {} because {}", failedRange, phase, nodeDisplayName, message);
+            failureHandler.addFailure(failedRange, instance, message);
+        });
+    }
+
+    public void failIfRingChanged()
+    {
+        if (monitor.getRingChanged())
+        {
+            throw new RuntimeException(String.format("Ring changed during %s stage of import. "
+                                                   + "Please rerun import once topology changes are complete.",
+                                                     phase));
+        }
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/BulkWriterContext.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/BulkWriterContext.java
new file mode 100644
index 0000000..e3b2421
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/BulkWriterContext.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.cassandra.spark.bulkwriter;
+
+import java.io.Serializable;
+
+public interface BulkWriterContext extends Serializable
+{
+    ClusterInfo cluster();
+
+    JobInfo job();
+
+    SchemaInfo schema();
+
+    DataTransferApi transfer();
+
+    // NOTE: This interface intentionally does *not* implement AutoClosable as Spark can close Broadcast variables
+    //       that implement AutoClosable while they are still in use, causing the underlying object to become unusable
+    void shutdown();
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CancelJobEvent.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CancelJobEvent.java
new file mode 100644
index 0000000..ab3119b
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CancelJobEvent.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.cassandra.spark.bulkwriter;
+
+public class CancelJobEvent
+{
+    public final Throwable exception;
+    public final String reason;
+
+    public CancelJobEvent(String reason)
+    {
+        this.reason = reason;
+        this.exception = null;
+    }
+
+    public CancelJobEvent(String reason, Throwable throwable)
+    {
+        this.reason = reason;
+        this.exception = throwable;
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CassandraBulkSourceRelation.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CassandraBulkSourceRelation.java
new file mode 100644
index 0000000..15d4371
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CassandraBulkSourceRelation.java
@@ -0,0 +1,185 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+import java.io.Serializable;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.api.java.JavaPairRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SQLContext;
+import org.apache.spark.sql.sources.BaseRelation;
+import org.apache.spark.sql.sources.InsertableRelation;
+import org.apache.spark.sql.types.StructType;
+import org.jetbrains.annotations.NotNull;
+import scala.Tuple2;
+import scala.collection.JavaConverters;
+import scala.util.control.NonFatal$;
+
+public class CassandraBulkSourceRelation extends BaseRelation implements InsertableRelation
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(CassandraBulkSourceRelation.class);
+
+    private final BulkWriterContext writerContext;
+    private final SQLContext sqlContext;
+    private final JavaSparkContext sparkContext;
+    private final Broadcast<BulkWriterContext> broadcastContext;
+    private final BulkWriteValidator writeValidator;
+
+    @SuppressWarnings("RedundantTypeArguments")
+    public CassandraBulkSourceRelation(BulkWriterContext writerContext, SQLContext sqlContext) throws Exception
+    {
+        this.writerContext = writerContext;
+        this.sqlContext = sqlContext;
+        this.sparkContext = JavaSparkContext.fromSparkContext(sqlContext.sparkContext());
+        this.broadcastContext = sparkContext.<BulkWriterContext>broadcast(writerContext);
+        this.writeValidator = new BulkWriteValidator(writerContext, this::cancelJob);
+    }
+
+    @Override
+    @NotNull
+    public SQLContext sqlContext()
+    {
+        return sqlContext;
+    }
+
+    /**
+     * @return An empty {@link StructType}, as this is a writer only, so schema is not applicable
+     */
+    @Override
+    @NotNull
+    public StructType schema()
+    {
+        LOGGER.warn("This instance is used as writer, a schema is not supported");
+        return new StructType();
+    }
+
+    /**
+     * @return {@code 0} size as not applicable use by the planner in the writer-only use case
+     */
+    @Override
+    public long sizeInBytes()
+    {
+        LOGGER.warn("This instance is used as writer, sizeInBytes is not supported");
+        return 0L;
+    }
+
+    @Override
+    public void insert(@NotNull Dataset<Row> data, boolean overwrite)
+    {
+        if (overwrite)
+        {
+            throw new LoadNotSupportedException("Overwriting existing data needs TRUNCATE on Cassandra, which is not supported");
+        }
+        writerContext.cluster().checkBulkWriterIsEnabledOrThrow();
+        Tokenizer tokenizer = new Tokenizer(writerContext);
+        TableSchema tableSchema = writerContext.schema().getTableSchema();
+        JavaPairRDD<DecoratedKey, Object[]> sortedRDD = data.toJavaRDD()
+                .map(Row::toSeq)
+                .map(seq -> JavaConverters.seqAsJavaListConverter(seq).asJava().toArray())
+                .map(tableSchema::normalize)
+                .keyBy(tokenizer::getDecoratedKey)
+                .repartitionAndSortWithinPartitions(broadcastContext.getValue().job().getTokenPartitioner());
+        persist(sortedRDD);
+    }
+
+    private void cancelJob(@NotNull CancelJobEvent cancelJobEvent)
+    {
+        if (cancelJobEvent.exception != null)
+        {
+            LOGGER.error("An unrecoverable error occurred during {} stage of import while validating the current cluster state; cancelling job",
+                         writeValidator.getPhase(), cancelJobEvent.exception);
+        }
+        else
+        {
+            LOGGER.error("Job was canceled due to '{}' during {} stage of import; please rerun import once topology changes are complete",
+                         cancelJobEvent.reason, writeValidator.getPhase());
+        }
+        sparkContext.cancelJobGroup(writerContext.job().getId().toString());
+    }
+
+    @SuppressWarnings("RedundantCast")
+    private void persist(@NotNull JavaPairRDD<DecoratedKey, Object[]> sortedRDD)
+    {
+        writeValidator.setPhase("Environment Validation");
+        writeValidator.validateInitialEnvironment();
+        writeValidator.setPhase("UploadAndCommit");
+
+        try
+        {
+            sortedRDD.foreachPartition(new WriteIterator(broadcastContext)::call);
+            writeValidator.failIfRingChanged();
+        }
+        catch (Throwable throwable)
+        {
+            LOGGER.error("Bulk Write Failed", throwable);
+            throw new RuntimeException("Bulk Write to Cassandra has failed", throwable);
+        }
+        finally
+        {
+            writeValidator.close();  // Uses the MgrClient, so needs to stop first
+            try
+            {
+                writerContext.shutdown();
+                sqlContext().sparkContext().clearJobGroup();
+            }
+            catch (Exception ignored)
+            {
+                // We've made our best effort to close the Bulk Writer context
+            }
+            try
+            {
+                broadcastContext.unpersist(false);
+            }
+            catch (Throwable throwable)
+            {
+                if (NonFatal$.MODULE$.apply(throwable))
+                {
+                    LOGGER.error("Uncaught exception in thread {} attempting to unpersist broadcast variable",
+                                 Thread.currentThread().getName(), throwable);
+                }
+                else
+                {
+                    throw throwable;
+                }
+            }
+        }
+    }
+
+    private static class WriteIterator implements Serializable
+    {
+        private final Broadcast<BulkWriterContext> broadcastContext;
+
+        WriteIterator(Broadcast<BulkWriterContext> broadcastContext)
+        {
+            this.broadcastContext = broadcastContext;
+        }
+
+        public void call(java.util.Iterator<Tuple2<DecoratedKey, Object[]>> iterator)
+        {
+            new RecordWriter(broadcastContext.getValue()).write(iterator);
+        }
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CassandraBulkWriterContext.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CassandraBulkWriterContext.java
new file mode 100644
index 0000000..4813b1c
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CassandraBulkWriterContext.java
@@ -0,0 +1,194 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+import java.util.Map;
+import java.util.Set;
+
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.KryoSerializable;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+import io.netty.channel.EventLoopGroup;
+import org.apache.cassandra.bridge.CassandraBridge;
+import org.apache.cassandra.bridge.CassandraBridgeFactory;
+import org.apache.cassandra.spark.bulkwriter.token.CassandraRing;
+import org.apache.cassandra.spark.data.CqlTable;
+import org.apache.cassandra.spark.data.ReplicationFactor;
+import org.apache.cassandra.spark.data.partitioner.Partitioner;
+import org.apache.cassandra.spark.utils.CqlUtils;
+import org.apache.cassandra.spark.utils.ScalaFunctions;
+import org.apache.spark.SparkContext;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.util.ShutdownHookManager;
+import org.jetbrains.annotations.NotNull;
+
+// CHECKSTYLE IGNORE: This class cannot be declared as final, because consumers should be able to extend it
+public class CassandraBulkWriterContext implements BulkWriterContext, KryoSerializable
+{
+    private static final long serialVersionUID = 8241993502687688783L;
+    private static final Logger LOGGER = LoggerFactory.getLogger(CassandraBulkWriterContext.class);
+
+    @NotNull
+    private final BulkSparkConf conf;
+    private final JobInfo jobInfo;
+    private final DataTransferApi dataTransferApi;
+    private final CassandraClusterInfo clusterInfo;
+    private final SchemaInfo schemaInfo;
+
+    static
+    {
+        configureRelocatedNetty();
+    }
+
+    private CassandraBulkWriterContext(@NotNull BulkSparkConf conf,
+                                       @NotNull StructType dfSchema,
+                                       SparkContext sparkContext)
+    {
+        this.conf = conf;
+        clusterInfo = new CassandraClusterInfo(conf);
+        CassandraRing<RingInstance> ring = clusterInfo.getRing(true);
+        jobInfo = new CassandraJobInfo(conf,
+                new TokenPartitioner(ring, conf.numberSplits, sparkContext.defaultParallelism(), conf.getCores()));
+        Preconditions.checkArgument(!conf.consistencyLevel.isLocal()
+                                 || (conf.localDC != null && ring.getReplicationFactor().getOptions().containsKey(conf.localDC)),
+                                    String.format("Keyspace %s is not replicated on datacenter %s",
+                                                  conf.keyspace, conf.localDC));
+
+        dataTransferApi = new SidecarDataTransferApi(clusterInfo.getCassandraContext().getSidecarClient(), jobInfo, conf);
+
+        String keyspace = conf.keyspace;
+        String table = conf.table;
+
+        String keyspaceSchema = clusterInfo.getKeyspaceSchema(true);
+        CassandraBridge bridge = CassandraBridgeFactory.get(clusterInfo.getLowestCassandraVersion());
+        Partitioner partitioner = clusterInfo.getPartitioner();
+        String tableSchema = CqlUtils.extractTableSchema(keyspaceSchema, keyspace, table);
+        Set<String> udts = CqlUtils.extractUdts(keyspaceSchema, keyspace);
+        ReplicationFactor replicationFactor = CqlUtils.extractReplicationFactor(keyspaceSchema, keyspace);
+        int indexCount = CqlUtils.extractIndexCount(keyspaceSchema, keyspace, table);
+        CqlTable cqlTable = bridge.buildSchema(tableSchema, keyspace, replicationFactor, partitioner, udts, null, indexCount);
+
+        TableInfoProvider tableInfoProvider = new CqlTableInfoProvider(tableSchema, cqlTable);
+        schemaInfo = new CassandraSchemaInfo(new TableSchema(dfSchema, tableInfoProvider, conf.writeMode));
+    }
+
+    public static BulkWriterContext fromOptions(@NotNull SparkContext sparkContext,
+                                                @NotNull Map<String, String> strOptions,
+                                                @NotNull StructType dfSchema)
+    {
+        Preconditions.checkNotNull(dfSchema);
+
+        BulkSparkConf conf = new BulkSparkConf(sparkContext.getConf(), strOptions);
+        CassandraBulkWriterContext bulkWriterContext = new CassandraBulkWriterContext(conf, dfSchema, sparkContext);
+        ShutdownHookManager.addShutdownHook(org.apache.spark.util.ShutdownHookManager.TEMP_DIR_SHUTDOWN_PRIORITY(),
+                                            ScalaFunctions.wrapLambda(bulkWriterContext::shutdown));
+        bulkWriterContext.dialHome(sparkContext.version());
+        return bulkWriterContext;
+    }
+
+    @Override
+    public void shutdown()
+    {
+        LOGGER.info("Shutting down {}", this);
+        synchronized (this)
+        {
+            if (clusterInfo != null)
+            {
+                clusterInfo.close();
+            }
+        }
+    }
+
+    // If using the shaded JAR, Configure our shaded Netty so it can find the correct native libraries
+    private static void configureRelocatedNetty()
+    {
+        if (EventLoopGroup.class.getName().startsWith("analytics"))
+        {
+            System.setProperty("analytics.io.netty.packagePrefix", "analytics");
+        }
+    }
+
+    /**
+     * Use the implementation of the KryoSerializable interface as a detection device to make sure the Spark Bulk
+     * Writer's KryoRegistrator is properly in place.
+     *
+     * If this class is serialized by Kryo, it means we're <b>not</b> set up correctly, and therefore we log and fail.
+     * This failure will occur early in the job and be very clear, so users can quickly fix their code and get up and
+     * running again, rather than having a random NullPointerException further down the line.
+     */
+    public static final String KRYO_REGISTRATION_WARNING =
+            "Spark Bulk Writer Kryo Registrator (SbwKryoRegistrator) was not registered with Spark - "
+          + "please see the README.md file for more details on how to register the Spark Bulk Writer.";
+
+    @Override
+    public void write(Kryo kryo, Output output)
+    {
+        failIfKryoNotRegistered();
+    }
+
+    @Override
+    public void read(Kryo kryo, Input input)
+    {
+        failIfKryoNotRegistered();
+    }
+
+    private void failIfKryoNotRegistered()
+    {
+        LOGGER.error(KRYO_REGISTRATION_WARNING);
+        throw new RuntimeException(KRYO_REGISTRATION_WARNING);
+    }
+
+    protected void dialHome(String sparkVersion)
+    {
+        LOGGER.info("Dial home. clientConfig={}, sparkVersion={}", conf, sparkVersion);
+    }
+
+    @Override
+    @NotNull
+    public ClusterInfo cluster()
+    {
+        return clusterInfo;
+    }
+
+    @Override
+    @NotNull
+    public JobInfo job()
+    {
+        return jobInfo;
+    }
+
+    @Override
+    public SchemaInfo schema()
+    {
+        return schemaInfo;
+    }
+
+    @Override
+    @NotNull
+    public DataTransferApi transfer()
+    {
+        return dataTransferApi;
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CassandraClusterInfo.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CassandraClusterInfo.java
new file mode 100644
index 0000000..e84957f
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CassandraClusterInfo.java
@@ -0,0 +1,499 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+import java.io.Closeable;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.bridge.CassandraVersionFeatures;
+import org.apache.cassandra.clients.Sidecar;
+import org.apache.cassandra.sidecar.client.SidecarInstance;
+import org.apache.cassandra.sidecar.client.SidecarInstanceImpl;
+import org.apache.cassandra.sidecar.common.NodeSettings;
+import org.apache.cassandra.sidecar.common.data.GossipInfoResponse;
+import org.apache.cassandra.sidecar.common.data.RingEntry;
+import org.apache.cassandra.sidecar.common.data.RingResponse;
+import org.apache.cassandra.sidecar.common.data.SchemaResponse;
+import org.apache.cassandra.sidecar.common.data.TimeSkewResponse;
+import org.apache.cassandra.spark.bulkwriter.token.CassandraRing;
+import org.apache.cassandra.spark.common.client.InstanceState;
+import org.apache.cassandra.spark.common.client.InstanceStatus;
+import org.apache.cassandra.spark.data.ReplicationFactor;
+import org.apache.cassandra.spark.data.partitioner.Partitioner;
+import org.apache.cassandra.spark.utils.CqlUtils;
+import org.jetbrains.annotations.NotNull;
+
+public class CassandraClusterInfo implements ClusterInfo, Closeable
+{
+    private static final long serialVersionUID = -6944818863462956767L;
+    private static final Logger LOGGER = LoggerFactory.getLogger(CassandraClusterInfo.class);
+
+    protected final BulkSparkConf conf;
+    protected String cassandraVersion;
+    protected Partitioner partitioner;
+    protected transient CassandraRing<RingInstance> ring;
+    protected transient String keyspaceSchema;
+    protected transient volatile RingResponse ringResponse;
+    protected transient GossipInfoResponse gossipInfo;
+    protected transient CassandraContext cassandraContext;
+    protected transient NodeSettings nodeSettings;
+
+    public CassandraClusterInfo(BulkSparkConf conf)
+    {
+        this.conf = conf;
+        this.cassandraContext = buildCassandraContext();
+    }
+
+    @Override
+    public void checkBulkWriterIsEnabledOrThrow()
+    {
+        // DO NOTHING
+    }
+
+    public String getVersion()
+    {
+        return CassandraClusterInfo.class.getPackage().getImplementationVersion();
+    }
+
+    @Override
+    public Map<RingInstance, InstanceAvailability> getInstanceAvailability()
+    {
+        RingResponse ringResponse = getRingResponse();
+        Map<RingInstance, InstanceAvailability> result = ringResponse
+                .stream()
+                .collect(Collectors.toMap(CassandraClusterInfo::getCasInstanceMethodsImpl,
+                                          this::determineInstanceAvailability));
+        if (LOGGER.isDebugEnabled())
+        {
+            result.forEach((instance, availability) ->
+                    LOGGER.debug("Instance {} has availability {}", instance, availability));
+        }
+        return result;
+    }
+
+    @Override
+    public boolean instanceIsAvailable(RingInstance ringInstance)
+    {
+        return instanceIsUp(ringInstance.getRingInstance())
+            && instanceIsNormal(ringInstance.getRingInstance())
+            && !instanceIsBlocked(ringInstance);
+    }
+
+    @Override
+    public InstanceState getInstanceState(RingInstance ringInstance)
+    {
+        return InstanceState.valueOf(ringInstance.getRingInstance().state().toUpperCase());
+    }
+
+    public CassandraContext getCassandraContext()
+    {
+        CassandraContext currentCassandraContext = cassandraContext;
+        if (currentCassandraContext != null)
+        {
+            return currentCassandraContext;
+        }
+
+        synchronized (this)
+        {
+            if (cassandraContext == null)
+            {
+                cassandraContext = buildCassandraContext();
+            }
+            return cassandraContext;
+        }
+    }
+
+    /**
+     * Gets a Cassandra Context
+     *
+     * NOTE: The caller of this method is required to call `shutdown` on the returned CassandraContext instance
+     *
+     * @return an instance of CassandraContext based on the configuration settings
+     */
+    protected CassandraContext buildCassandraContext()
+    {
+        return buildCassandraContext(conf);
+    }
+
+    private static CassandraContext buildCassandraContext(BulkSparkConf conf)
+    {
+        return CassandraContext.create(conf);
+    }
+
+    @Override
+    public void close()
+    {
+        synchronized (this)
+        {
+            LOGGER.info("Closing {}", this);
+            getCassandraContext().close();
+        }
+    }
+
+    @Override
+    public Partitioner getPartitioner()
+    {
+        Partitioner currentPartitioner = partitioner;
+        if (currentPartitioner != null)
+        {
+            return currentPartitioner;
+        }
+
+        synchronized (this)
+        {
+            if (partitioner == null)
+            {
+                try
+                {
+                    String partitionerString;
+                    NodeSettings currentNodeSettings = nodeSettings;
+                    if (currentNodeSettings != null)
+                    {
+                        partitionerString = currentNodeSettings.partitioner();
+                    }
+                    else
+                    {
+                        partitionerString = getCassandraContext().getSidecarClient().nodeSettings().get().partitioner();
+                    }
+                    partitioner = Partitioner.from(partitionerString);
+                }
+                catch (ExecutionException | InterruptedException exception)
+                {
+                    throw new RuntimeException("Unable to retrieve partitioner information", exception);
+                }
+            }
+            return partitioner;
+        }
+    }
+
+    @Override
+    public TimeSkewResponse getTimeSkew(List<RingInstance> replicas)
+    {
+        try
+        {
+            List<SidecarInstance> instances = replicas
+                    .stream()
+                    .map(replica -> new SidecarInstanceImpl(replica.getNodeName(), conf.getSidecarPort()))
+                    .collect(Collectors.toList());
+            return getCassandraContext().getSidecarClient().timeSkew(instances).get();
+        }
+        catch (InterruptedException | ExecutionException exception)
+        {
+            throw new RuntimeException(exception);
+        }
+    }
+
+    @Override
+    public void refreshClusterInfo()
+    {
+        synchronized (this)
+        {
+            // Set backing stores to null and let them lazy-load on the next call
+            ringResponse = null;
+            gossipInfo = null;
+            keyspaceSchema = null;
+            getCassandraContext().refreshClusterConfig();
+        }
+    }
+
+    protected String getCurrentKeyspaceSchema() throws Exception
+    {
+        SchemaResponse schemaResponse = getCassandraContext().getSidecarClient().schema(conf.keyspace).get();
+        return schemaResponse.schema();
+    }
+
+    @NotNull
+    protected CassandraRing<RingInstance> getCurrentRing() throws Exception
+    {
+        RingResponse ringResponse = getCurrentRingResponse();
+        List<RingInstance> instances = getSerializableInstances(ringResponse);
+        ReplicationFactor replicationFactor = getReplicationFactor();
+        return new CassandraRing<>(getPartitioner(), conf.keyspace, replicationFactor, instances);
+    }
+
+    @NotNull
+    protected ReplicationFactor getReplicationFactor()
+    {
+        String keyspaceSchema = getKeyspaceSchema(true);
+        if (keyspaceSchema == null)
+        {
+            throw new RuntimeException(String.format("Could not keyspace schema information for keyspace %s",
+                                                     conf.keyspace));
+        }
+        return CqlUtils.extractReplicationFactor(keyspaceSchema, conf.keyspace);
+    }
+
+    @Override
+    public String getKeyspaceSchema(boolean cached)
+    {
+        String currentKeyspaceSchema = keyspaceSchema;
+        if (cached && currentKeyspaceSchema != null)
+        {
+            return currentKeyspaceSchema;
+        }
+
+        synchronized (this)
+        {
+            if (!cached || keyspaceSchema == null)
+            {
+                try
+                {
+                    keyspaceSchema = getCurrentKeyspaceSchema();
+                }
+                catch (Exception exception)
+                {
+                    throw new RuntimeException("Unable to initialize schema information for keyspace " + conf.keyspace,
+                                               exception);
+                }
+            }
+            return keyspaceSchema;
+        }
+    }
+
+    @Override
+    public CassandraRing<RingInstance> getRing(boolean cached)
+    {
+        CassandraRing<RingInstance> currentRing = ring;
+        if (cached && currentRing != null)
+        {
+            return currentRing;
+        }
+
+        synchronized (this)
+        {
+            if (!cached || ring == null)
+            {
+                try
+                {
+                    ring = getCurrentRing();
+                }
+                catch (Exception exception)
+                {
+                    throw new RuntimeException("Unable to initialize ring information", exception);
+                }
+            }
+            return ring;
+        }
+    }
+
+    @Override
+    public String getLowestCassandraVersion()
+    {
+        String currentCassandraVersion = cassandraVersion;
+        if (currentCassandraVersion != null)
+        {
+            return currentCassandraVersion;
+        }
+
+        synchronized (this)
+        {
+            if (cassandraVersion == null)
+            {
+                String versionFromFeature = getVersionFromFeature();
+                if (versionFromFeature != null)
+                {
+                    // Forcing writer to use a particular version
+                    cassandraVersion = versionFromFeature;
+                }
+                else
+                {
+                    cassandraVersion = getVersionFromSidecar();
+                }
+            }
+        }
+        return cassandraVersion;
+    }
+
+    public String getVersionFromFeature()
+    {
+        return null;
+    }
+
+    public String getVersionFromSidecar()
+    {
+        LOGGER.info("Getting Cassandra versions from all nodes");
+        List<NodeSettings> allNodeSettings = Sidecar.allNodeSettingsBlocking(conf,
+                                                                             cassandraContext.getSidecarClient(),
+                                                                             cassandraContext.clusterConfig);
+        return getLowestVersion(allNodeSettings);
+    }
+
+    protected RingResponse getRingResponse()
+    {
+        RingResponse currentRingResponse = ringResponse;
+        if (currentRingResponse != null)
+        {
+            return currentRingResponse;
+        }
+
+        synchronized (this)
+        {
+            if (ringResponse == null)
+            {
+                try
+                {
+                    ringResponse = getCurrentRingResponse();
+                }
+                catch (Exception exception)
+                {
+                    LOGGER.error("Failed to load Cassandra ring", exception);
+                    throw new RuntimeException(exception);
+                }
+            }
+            return ringResponse;
+        }
+    }
+
+    private RingResponse getCurrentRingResponse() throws Exception
+    {
+        return getCassandraContext().getSidecarClient().ring(conf.keyspace).get();
+    }
+
+    private static List<RingInstance> getSerializableInstances(RingResponse ringResponse)
+    {
+        return ringResponse.stream()
+                           .map(RingInstance::new)
+                           .collect(Collectors.toList());
+    }
+
+    private static RingInstance getCasInstanceMethodsImpl(RingEntry ringEntry)
+    {
+        return new RingInstance(ringEntry);
+    }
+
+    protected GossipInfoResponse getGossipInfo(boolean forceRefresh)
+    {
+        GossipInfoResponse currentGossipInfo = gossipInfo;
+        if (!forceRefresh && currentGossipInfo != null)
+        {
+            return currentGossipInfo;
+        }
+
+        synchronized (this)
+        {
+            if (forceRefresh || gossipInfo == null)
+            {
+                try
+                {
+                    gossipInfo = cassandraContext.getSidecarClient().gossipInfo().get(conf.getHttpResponseTimeoutMs(),
+                                                                                      TimeUnit.MILLISECONDS);
+                }
+                catch (ExecutionException | InterruptedException exception)
+                {
+                    LOGGER.error("Failed to retrieve gossip information");
+                    throw new RuntimeException("Failed to retrieve gossip information", exception);
+                }
+                catch (TimeoutException exception)
+                {
+                    Thread.currentThread().interrupt();
+                    throw new RuntimeException("Failed to retrieve gossip information", exception);
+                }
+            }
+            return gossipInfo;
+        }
+    }
+
+    private InstanceAvailability determineInstanceAvailability(RingEntry ringEntry)
+    {
+        if (!instanceIsUp(ringEntry))
+        {
+            return InstanceAvailability.UNAVAILABLE_DOWN;
+        }
+        else if (instanceIsJoining(ringEntry) && isReplacement(ringEntry))
+        {
+            return InstanceAvailability.UNAVAILABLE_REPLACEMENT;
+        }
+        else if (instanceIsBlocked(getCasInstanceMethodsImpl(ringEntry)))
+        {
+            return InstanceAvailability.UNAVAILABLE_BLOCKED;
+        }
+        else if (instanceIsNormal(ringEntry))
+        {
+            return InstanceAvailability.AVAILABLE;
+        }
+        else
+        {
+            // If it's not one of the above, it's inherently INVALID
+            return InstanceAvailability.INVALID_STATE;
+        }
+    }
+
+    @VisibleForTesting
+    public String getLowestVersion(List<NodeSettings> allNodeSettings)
+    {
+        nodeSettings = allNodeSettings
+                .stream()
+                .filter(settings -> !settings.releaseVersion().equalsIgnoreCase("unknown"))
+                .min(Comparator.comparing(settings ->
+                        CassandraVersionFeatures.cassandraVersionFeaturesFromCassandraVersion(settings.releaseVersion())))
+                .orElseThrow(() -> new RuntimeException("No valid Cassandra Versions were returned from Cassandra Sidecar"));
+        return nodeSettings.releaseVersion();
+    }
+
+    protected boolean instanceIsBlocked(RingInstance ignored)
+    {
+        return false;
+    }
+
+    protected boolean instanceIsNormal(RingEntry ringEntry)
+    {
+        return InstanceState.NORMAL.name().equalsIgnoreCase(ringEntry.state());
+    }
+
+    protected boolean instanceIsUp(RingEntry ringEntry)
+    {
+        return InstanceStatus.UP.name().equalsIgnoreCase(ringEntry.status());
+    }
+
+    protected boolean instanceIsJoining(RingEntry ringEntry)
+    {
+        return InstanceState.JOINING.name().equalsIgnoreCase(ringEntry.state());
+    }
+
+    protected boolean isReplacement(RingEntry ringEntry)
+    {
+        GossipInfoResponse gossipInfo = getGossipInfo(true);
+        LOGGER.debug("Gossip info={}", gossipInfo);
+
+        GossipInfoResponse.GossipInfo hostInfo = gossipInfo.get(ringEntry.address());
+        if (hostInfo != null)
+        {
+            LOGGER.info("Found hostinfo: {}", hostInfo);
+            String hostStatus = hostInfo.status();
+            if (hostStatus != null)
+            {
+                // If status has gone to NORMAL, we can't determine here if this was a host replacement or not.
+                // CassandraRingManager will handle detecting the ring change if it's gone NORMAL after the job starts.
+                return hostStatus.startsWith("BOOT_REPLACE,") || hostStatus.equals("NORMAL");
+            }
+        }
+        return false;
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CassandraContext.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CassandraContext.java
new file mode 100644
index 0000000..e5b6853
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CassandraContext.java
@@ -0,0 +1,99 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+import java.io.Closeable;
+import java.util.ArrayList;
+import java.util.Set;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.clients.Sidecar;
+import org.apache.cassandra.sidecar.client.SidecarClient;
+import org.apache.cassandra.sidecar.client.SidecarInstance;
+import org.apache.cassandra.sidecar.client.SimpleSidecarInstancesProvider;
+import org.jetbrains.annotations.NotNull;
+
+public class CassandraContext implements Closeable
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(CassandraContext.class);
+    @NotNull
+    protected Set<? extends SidecarInstance> clusterConfig;
+    private final BulkSparkConf conf;
+    private final transient SidecarClient sidecarClient;
+
+    protected CassandraContext(BulkSparkConf conf)
+    {
+        this.conf = conf;
+        this.clusterConfig = createClusterConfig();
+        this.sidecarClient = initializeSidecarClient(conf);
+        LOGGER.debug("[{}] Created Cassandra Context", Thread.currentThread().getName());
+    }
+
+    public static CassandraContext create(BulkSparkConf conf)
+    {
+        return new CassandraContext(conf);
+    }
+
+    public Set<? extends SidecarInstance> getCluster()
+    {
+        return clusterConfig;
+    }
+
+    public void refreshClusterConfig()
+    {
+        // DO NOTHING
+    }
+
+    @Override
+    public void close()
+    {
+        try
+        {
+            sidecarClient.close();
+            LOGGER.debug("[{}] Closed Cassandra Context", Thread.currentThread().getName());
+        }
+        catch (Throwable throwable)
+        {
+            LOGGER.error("Could not shut down CassandraContext.", throwable);
+        }
+    }
+
+    protected SidecarClient initializeSidecarClient(BulkSparkConf conf)
+    {
+        return Sidecar.from(new SimpleSidecarInstancesProvider(new ArrayList<>(clusterConfig)), conf);
+    }
+
+    protected Set<? extends SidecarInstance> createClusterConfig()
+    {
+        return conf.sidecarInstances;
+    }
+
+    public SidecarClient getSidecarClient()
+    {
+        return sidecarClient;
+    }
+
+    protected BulkSparkConf conf()
+    {
+        return conf;
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CassandraJobInfo.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CassandraJobInfo.java
new file mode 100644
index 0000000..3151d6e
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CassandraJobInfo.java
@@ -0,0 +1,119 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+import java.util.UUID;
+
+import org.apache.cassandra.spark.bulkwriter.token.ConsistencyLevel;
+import org.jetbrains.annotations.NotNull;
+
+public class CassandraJobInfo implements JobInfo
+{
+    private static final long serialVersionUID = 6140098484732683759L;
+    private final BulkSparkConf conf;
+    @NotNull
+    private final UUID jobId = UUID.randomUUID();
+    private final TokenPartitioner tokenPartitioner;
+
+    CassandraJobInfo(BulkSparkConf conf, TokenPartitioner tokenPartitioner)
+    {
+        this.conf = conf;
+        this.tokenPartitioner = tokenPartitioner;
+    }
+
+    @Override
+    public ConsistencyLevel getConsistencyLevel()
+    {
+        return conf.consistencyLevel;
+    }
+
+    @Override
+    public String getLocalDC()
+    {
+        return conf.localDC;
+    }
+
+    @Override
+    public RowBufferMode getRowBufferMode()
+    {
+        return conf.rowBufferMode;
+    }
+
+    @Override
+    public int getSstableDataSizeInMB()
+    {
+        return conf.sstableDataSizeInMB;
+    }
+
+    @Override
+    public int getSstableBatchSize()
+    {
+        return conf.sstableBatchSize;
+    }
+
+    @Override
+    public int getCommitBatchSize()
+    {
+        return conf.commitBatchSize;
+    }
+
+    @Override
+    public boolean validateSSTables()
+    {
+        return conf.validateSSTables;
+    }
+
+    @Override
+    public boolean skipExtendedVerify()
+    {
+        return conf.skipExtendedVerify;
+    }
+
+    @Override
+    public boolean getSkipClean()
+    {
+        return conf.getSkipClean();
+    }
+
+    @Override
+    public int getCommitThreadsPerInstance()
+    {
+        return conf.commitThreadsPerInstance;
+    }
+
+    @Override
+    public UUID getId()
+    {
+        return jobId;
+    }
+
+    @Override
+    public TokenPartitioner getTokenPartitioner()
+    {
+        return tokenPartitioner;
+    }
+
+    @Override
+    @NotNull
+    public String getFullTableName()
+    {
+        return conf.keyspace + "." + conf.table;
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CassandraRingMonitor.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CassandraRingMonitor.java
new file mode 100644
index 0000000..ead2cc3
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CassandraRingMonitor.java
@@ -0,0 +1,135 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+import java.math.BigInteger;
+import java.util.Collection;
+import java.util.concurrent.Callable;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Range;
+import org.apache.commons.collections.CollectionUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.spark.bulkwriter.token.CassandraRing;
+import org.apache.cassandra.spark.bulkwriter.util.ThreadUtil;
+import org.apache.cassandra.spark.common.client.ClientException;
+
+public class CassandraRingMonitor
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(CassandraRingMonitor.class);
+
+    private CassandraRing<RingInstance> initialRing;
+    private volatile boolean ringChanged = false;
+    private Callable<CassandraRing<RingInstance>> ringSupplier;
+    private Consumer<CancelJobEvent> changeFunc;
+    private final ScheduledExecutorService executorService;
+
+    public CassandraRingMonitor(ClusterInfo clusterInfo,
+                                Consumer<CancelJobEvent> changeFunc,
+                                int checkInterval,
+                                TimeUnit checkIntervalUnit) throws Exception
+    {
+        this(() -> clusterInfo.getRing(false),
+             changeFunc,
+             checkInterval,
+             checkIntervalUnit,
+             Executors.newSingleThreadScheduledExecutor(ThreadUtil.threadFactory("Cassandra Ring Monitor")));
+    }
+
+    @VisibleForTesting
+    public CassandraRingMonitor(Callable<CassandraRing<RingInstance>> ringSupplier,
+                                Consumer<CancelJobEvent> changeFunc,
+                                int checkInterval,
+                                TimeUnit checkIntervalUnit,
+                                ScheduledExecutorService executorService) throws Exception
+    {
+        this.ringSupplier = ringSupplier;
+        this.initialRing = ringSupplier.call();
+        this.changeFunc = changeFunc;
+        this.executorService = executorService;
+        executorService.scheduleAtFixedRate(this::checkRingChanged, 0, checkInterval, checkIntervalUnit);
+    }
+
+    /**
+     * Reads the ring changed value from the monitor. This actual ring
+     * is retrieved and compared every second - this is just the backing
+     * store for the result.
+     *
+     * @return if the ring has changed or not since this instance was created
+     */
+    public boolean getRingChanged()
+    {
+        return ringChanged;
+    }
+
+    public void stop()
+    {
+        executorService.shutdown();
+        try
+        {
+            executorService.awaitTermination(2, TimeUnit.SECONDS);
+        }
+        catch (InterruptedException exception)
+        {
+            // Do nothing as we're just waiting for the ring call to complete
+        }
+    }
+
+    private void checkRingChanged()
+    {
+        LOGGER.debug("Checking ring for changes");
+        try
+        {
+            CassandraRing<RingInstance> currentRing = ringSupplier.call();
+            if (checkRingChanged(currentRing))
+            {
+                ringChanged = true;
+                changeFunc.accept(new CancelJobEvent("Ring changed during bulk load"));
+                executorService.shutdownNow();
+            }
+        }
+        catch (ClientException exception)
+        {
+            LOGGER.warn("Could not retrieve current ring. Will retry momentarily. Continuing bulk load.", exception);
+        }
+        catch (Exception exception)
+        {
+            String message = "Error while attempting to determine if ring changed. Failing job";
+            LOGGER.error(message, exception);
+            changeFunc.accept(new CancelJobEvent(message, exception));
+            executorService.shutdownNow();
+        }
+    }
+
+    private boolean checkRingChanged(CassandraRing<RingInstance> currentRing)
+    {
+        Collection<Range<BigInteger>> currentTokenRanges = currentRing.getTokenRanges().values();
+        Collection<Range<BigInteger>> startingTokenRanges = initialRing.getTokenRanges().values();
+        LOGGER.debug("Current token range: {}", currentTokenRanges);
+        LOGGER.debug("Initial token range: {}", startingTokenRanges);
+        return !CollectionUtils.isEqualCollection(currentTokenRanges, startingTokenRanges);
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CassandraSchemaInfo.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CassandraSchemaInfo.java
new file mode 100644
index 0000000..d55b49b
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CassandraSchemaInfo.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.cassandra.spark.bulkwriter;
+
+public class CassandraSchemaInfo implements SchemaInfo
+{
+    private static final long serialVersionUID = -2327383232935001862L;
+    private final TableSchema tableSchema;
+
+    public CassandraSchemaInfo(TableSchema tableSchema)
+    {
+        this.tableSchema = tableSchema;
+    }
+
+    @Override
+    public TableSchema getTableSchema()
+    {
+        return tableSchema;
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/ClusterInfo.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/ClusterInfo.java
new file mode 100644
index 0000000..5b65a55
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/ClusterInfo.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.cassandra.spark.bulkwriter;
+
+import java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.cassandra.sidecar.common.data.TimeSkewResponse;
+import org.apache.cassandra.spark.bulkwriter.token.CassandraRing;
+import org.apache.cassandra.spark.common.client.InstanceState;
+import org.apache.cassandra.spark.data.partitioner.Partitioner;
+
+public interface ClusterInfo extends Serializable
+{
+    void refreshClusterInfo();
+
+    CassandraRing<RingInstance> getRing(boolean cached);
+
+    String getLowestCassandraVersion();
+
+    Map<RingInstance, InstanceAvailability> getInstanceAvailability();
+
+    boolean instanceIsAvailable(RingInstance ringInstance);
+
+    InstanceState getInstanceState(RingInstance instance);
+
+    Partitioner getPartitioner();
+
+    void checkBulkWriterIsEnabledOrThrow();
+
+    TimeSkewResponse getTimeSkew(List<RingInstance> replicas);
+
+    String getKeyspaceSchema(boolean cached);
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CommitCoordinator.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CommitCoordinator.java
new file mode 100644
index 0000000..2892a12
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CommitCoordinator.java
@@ -0,0 +1,223 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+import java.math.BigInteger;
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Range;
+import com.google.common.util.concurrent.AbstractFuture;
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.MoreExecutors;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.spark.bulkwriter.util.ThreadUtil;
+import org.jetbrains.annotations.Nullable;
+
+public final class CommitCoordinator extends AbstractFuture<List<CommitResult>> implements AutoCloseable
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(CommitCoordinator.class);
+
+    private final HashMap<RingInstance, ListeningExecutorService> executors = new HashMap<>();
+    private final List<StreamResult> successfulUploads;
+    private final DataTransferApi transferApi;
+    private final ClusterInfo cluster;
+    private final JobInfo job;
+    private ListenableFuture<List<CommitResult>> allCommits;
+    private final String jobSufix;
+
+    public static CommitCoordinator commit(BulkWriterContext bulkWriterContext, StreamResult[] uploadResults)
+    {
+        CommitCoordinator coordinator = new CommitCoordinator(bulkWriterContext, uploadResults);
+        coordinator.commit();
+        return coordinator;
+    }
+
+    private CommitCoordinator(BulkWriterContext writerContext, StreamResult[] uploadResults)
+    {
+        this.transferApi = writerContext.transfer();
+        this.cluster = writerContext.cluster();
+        this.job = writerContext.job();
+        this.jobSufix = "-" + job.getId();
+        successfulUploads = Arrays.stream(uploadResults)
+                                  .filter(result -> !result.passed.isEmpty())
+                                  .collect(Collectors.toList());
+    }
+
+    @Override
+    public boolean cancel(boolean mayInterruptIfRunning)
+    {
+        close();
+        return allCommits == null || allCommits.cancel(mayInterruptIfRunning);
+    }
+
+    void commit()
+    {
+        // We may have already committed - we should never get here if we did, but if somehow we do we should
+        // simply return the commit results we already collected
+        if (successfulUploads.size() > 0 && successfulUploads.stream()
+                                                             .allMatch(result -> result.commitResults != null
+                                                                              && result.commitResults.size() > 0))
+        {
+            List<CommitResult> collect = successfulUploads.stream()
+                    .flatMap(streamResult -> streamResult.commitResults.stream())
+                    .collect(Collectors.toList());
+            set(collect);
+            return;
+        }
+        // First, group commits by instance so we can multi-commit
+        Map<RingInstance, Map<String, Range<BigInteger>>> resultsByInstance = getResultsByInstance(successfulUploads);
+        List<ListenableFuture<CommitResult>> commitFutures = resultsByInstance.entrySet().stream()
+                .flatMap(entry -> commit(executors, entry.getKey(), entry.getValue()))
+                .collect(Collectors.toList());
+        // Create an aggregate ListenableFuture around the list of futures containing the results of the commit calls.
+        // We'll fail fast if any of those errMsg (note that an errMsg here means an unexpected exception,
+        // not a failure response from CassandraManager).
+        // The callback on the aggregate listener sets the return value for this AbstractFuture
+        // so callers can make blocking calls to CommitCoordinator::get.
+        allCommits = Futures.allAsList(commitFutures);
+        Futures.addCallback(allCommits,
+                            new FutureCallback<List<CommitResult>>()
+                            {
+                                public void onSuccess(@Nullable List<CommitResult> result)
+                                {
+                                    set(result);
+                                }
+
+                                public void onFailure(Throwable throwable)
+                                {
+                                    setException(throwable);
+                                }
+                            },
+                            Runnable::run);
+    }
+
+    private Stream<ListenableFuture<CommitResult>> commit(Map<RingInstance, ListeningExecutorService> executors,
+                                                          RingInstance instance,
+                                                          Map<String, Range<BigInteger>> uploadRanges)
+    {
+        if (cluster.instanceIsAvailable(instance))
+        {
+            ListeningExecutorService executorService = executors.computeIfAbsent(instance,
+                    key -> MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(
+                            job.getCommitThreadsPerInstance(),
+                            ThreadUtil.threadFactory("commit-sstable-" + key.getNodeName()))));
+            List<String> allUuids = new ArrayList<>(uploadRanges.keySet());
+            LOGGER.info("Committing UUIDs={}, Ranges={}, instance={}",
+                        allUuids, uploadRanges.values(), instance.getNodeName());
+            List<List<String>> batches = Lists.partition(allUuids, job.getCommitBatchSize());
+            return batches.stream().map(uuids -> {
+                String migrationId = UUID.randomUUID().toString();
+                return executorService.submit(() -> {
+                    CommitResult commitResult = new CommitResult(migrationId, instance, uploadRanges);
+                    try
+                    {
+                        DataTransferApi.RemoteCommitResult result = transferApi.commitSSTables(instance, migrationId, uuids);
+                        if (result.isSuccess)
+                        {
+                            LOGGER.info("[{}]: Commit succeeded on {} for {}",
+                                        migrationId, instance.getNodeName(), uploadRanges);
+                        }
+                        else
+                        {
+                            LOGGER.error("[{}]: Commit failed: uploadRanges: {}, failedUuids: {}, stdErr: {}",
+                                         migrationId,  uploadRanges.entrySet(), result.failedUuids, result.stdErr);
+                            if (result.failedUuids.size() > 0)
+                            {
+                                addFailures(result.failedUuids, uploadRanges, commitResult, result.stdErr);
+                            }
+                            else
+                            {
+                                addFailures(uploadRanges, commitResult, result.stdErr);
+                            }
+                        }
+                    }
+                    catch (Throwable throwable)
+                    {
+                        addFailures(uploadRanges, commitResult, throwable.toString());
+                        // On errMsg, refresh cluster information so we get the latest block list and status information to react accordingly
+                        cluster.refreshClusterInfo();
+                    }
+                    return commitResult;
+                });
+            });
+        }
+        else
+        {
+            String migrationId = UUID.randomUUID().toString();
+            CommitResult commitResult = new CommitResult(migrationId, instance, uploadRanges);
+
+            String message = String.format("Instance %s is not available.", instance.getNodeName());
+            addFailures(uploadRanges, commitResult, message);
+            return Stream.of(Futures.immediateFuture(commitResult));
+        }
+    }
+
+    private void addFailures(List<String> failedRanges,
+                             Map<String, Range<BigInteger>> uploadRanges,
+                             CommitResult commitResult,
+                             String error)
+    {
+        failedRanges.forEach(uuid -> {
+            String shortUuid = uuid.replace(jobSufix, "");
+            commitResult.addFailedCommit(shortUuid, uploadRanges.get(shortUuid), error != null ? error : "Unknown Commit Failure");
+        });
+    }
+
+    private void addFailures(Map<String, Range<BigInteger>> failedRanges, CommitResult commitResult, String message)
+    {
+        failedRanges.forEach((key, value) -> commitResult.addFailedCommit(key, value, message));
+        LOGGER.debug("Added failures to commitResult by Range: {}", commitResult);
+    }
+
+    private Map<RingInstance, Map<String, Range<BigInteger>>> getResultsByInstance(List<StreamResult> successfulUploads)
+    {
+        return successfulUploads
+                .stream()
+                .flatMap(upload -> upload.passed
+                        .stream()
+                        .map(instance -> new AbstractMap.SimpleEntry<>(instance,
+                                         new AbstractMap.SimpleEntry<>(upload.sessionID, upload.tokenRange))))
+                .collect(Collectors.groupingBy(AbstractMap.SimpleEntry::getKey,
+                         Collectors.toMap(instance -> instance.getValue().getKey(),
+                                          instance -> instance.getValue().getValue())));
+    }
+
+    @Override
+    public void close()
+    {
+        executors.values().forEach(ExecutorService::shutdownNow);
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CommitError.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CommitError.java
new file mode 100644
index 0000000..62dcc02
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CommitError.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.cassandra.spark.bulkwriter;
+
+import java.io.Serializable;
+import java.math.BigInteger;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Range;
+
+import org.jetbrains.annotations.NotNull;
+
+public class CommitError implements Serializable
+{
+    protected final Range<BigInteger> tokenRange;
+    protected final String errMsg;
+
+    public CommitError(@NotNull Range<BigInteger> tokenRange, @NotNull String errMsg)
+    {
+        Preconditions.checkNotNull(tokenRange, "CommitError created without a token range");
+        Preconditions.checkNotNull(errMsg, "CommitError created without an error message");
+        this.tokenRange = tokenRange;
+        this.errMsg = errMsg;
+    }
+
+    @Override
+    public String toString()
+    {
+        return String.format("CommitError{tokenRange=%s, errMsg='%s'}", tokenRange, errMsg);
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CommitResult.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CommitResult.java
new file mode 100644
index 0000000..6020d05
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CommitResult.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.cassandra.spark.bulkwriter;
+
+import java.io.Serializable;
+import java.math.BigInteger;
+import java.util.HashMap;
+import java.util.Map;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Range;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.jetbrains.annotations.NotNull;
+
+public class CommitResult implements Serializable
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(StreamResult.class);
+
+    public final String migrationId;
+    protected RingInstance instance;
+    public final Map<String, Range<BigInteger>> passed;
+    public final Map<String, CommitError> failures;
+
+    public CommitResult(String migrationId, RingInstance instance, Map<String, Range<BigInteger>> commitRanges)
+    {
+        this.migrationId = migrationId;
+        this.instance = instance;
+        this.passed = new HashMap<>(commitRanges);
+        this.failures = new HashMap<>();
+    }
+
+    public void addFailedCommit(String uuid, @NotNull Range<BigInteger> tokenRange, @NotNull String error)
+    {
+        Preconditions.checkNotNull(uuid, "Adding failed commit with missing UUID");
+        Preconditions.checkNotNull(tokenRange, "Adding failed commit with missing token range");
+        Preconditions.checkNotNull(error, "Adding failed commit with missing error message");
+        LOGGER.error("[{}]: Failed to commit {} on {}: {}", uuid, tokenRange, instance, error);
+        passed.remove(uuid);
+        failures.put(uuid, new CommitError(tokenRange, error));
+    }
+
+    @Override
+    public String toString()
+    {
+        return String.format("CommitResult{migrationId='%s', instance=%s, passed=%s, failures=%s}", migrationId, instance, passed, failures);
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CqlTableInfoProvider.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CqlTableInfoProvider.java
new file mode 100644
index 0000000..e512d27
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/CqlTableInfoProvider.java
@@ -0,0 +1,175 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.spark.common.schema.ColumnType;
+import org.apache.cassandra.spark.common.schema.ColumnTypes;
+import org.apache.cassandra.spark.data.CqlField;
+import org.apache.cassandra.spark.data.CqlTable;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.ASCII;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.BIGINT;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.BLOB;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.BOOLEAN;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.DOUBLE;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.INT;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.TEXT;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.TIMESTAMP;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.TIMEUUID;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.UUID;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.VARCHAR;
+
+/**
+ * An implementation of the {@link TableInfoProvider} interface that leverages the {@link CqlTable} to
+ * provide table information
+ */
+public class CqlTableInfoProvider implements TableInfoProvider
+{
+    // CHECKSTYLE IGNORE: Long line
+    private static final Pattern DEPRECATED_OPTIONS  = Pattern.compile("(?:\\s+AND)?\\s+(?:dclocal_)?read_repair_chance\\s*=\\s*[+\\-.\\dE]+", Pattern.CASE_INSENSITIVE);
+    private static final Pattern LEADING_CONJUNCTION = Pattern.compile("(?<=\\bWITH)\\s+AND\\b", Pattern.CASE_INSENSITIVE);
+
+    private final String createTableStatement;
+    private final CqlTable cqlTable;
+
+    /**
+     * Mapping from CQL Type Names to Java types
+     */
+    private static final Map<String, ColumnType<?>> DATA_TYPES = ImmutableMap.<String, ColumnType<?>>builder()
+                                                                             .put(BIGINT, ColumnTypes.LONG)
+                                                                             .put(BLOB, ColumnTypes.BYTES)
+                                                                             .put(DOUBLE, ColumnTypes.DOUBLE)
+                                                                             .put(INT, ColumnTypes.INT)
+                                                                             .put(BOOLEAN, ColumnTypes.BOOLEAN)
+                                                                             .put(TEXT, ColumnTypes.STRING)
+                                                                             .put(TIMESTAMP, ColumnTypes.LONG)
+                                                                             .put(UUID, ColumnTypes.UUID)
+                                                                             .put(VARCHAR, ColumnTypes.STRING)
+                                                                             .put(ASCII, ColumnTypes.STRING)
+                                                                             .put(TIMEUUID, ColumnTypes.UUID)
+                                                                             .build();
+
+    public CqlTableInfoProvider(String createTableStatement, CqlTable cqlTable)
+    {
+        this.createTableStatement = createTableStatement;
+        this.cqlTable = cqlTable;
+    }
+
+    @Override
+    public CqlField.CqlType getColumnType(String columnName)
+    {
+        CqlField cqlField = cqlTable.column(columnName);
+        if (cqlField == null)
+        {
+            throw new IllegalArgumentException("Unknown fields in data frame => " + columnName);
+        }
+        else
+        {
+            return cqlField.type();
+        }
+    }
+
+    @Override
+    public List<ColumnType<?>> getPartitionKeyTypes()
+    {
+        return cqlTable.partitionKeys().stream()
+                       .map(cqlField -> DATA_TYPES.get(cqlField.type().cqlName().toLowerCase()))
+                       .collect(Collectors.toList());
+    }
+
+    @Override
+    public boolean columnExists(String columnName)
+    {
+        return cqlTable.column(columnName) != null;
+    }
+
+    @Override
+    public List<String> getPartitionKeyColumnNames()
+    {
+        return cqlTable.partitionKeys().stream()
+                       .map(CqlField::name)
+                       .collect(Collectors.toList());
+    }
+
+    @Override
+    public String getCreateStatement()
+    {
+        return removeDeprecatedOptions(createTableStatement);
+    }
+
+    @Override
+    public List<String> getPrimaryKeyColumnNames()
+    {
+        return cqlTable.primaryKey().stream()
+                       .map(CqlField::name)
+                       .collect(Collectors.toList());
+    }
+
+    @Override
+    public String getName()
+    {
+        return cqlTable.table();
+    }
+
+    @Override
+    public String getKeyspaceName()
+    {
+        return cqlTable.keyspace();
+    }
+
+    @Override
+    public boolean hasSecondaryIndex()
+    {
+        return cqlTable.indexCount() > 0;
+    }
+
+    @Override
+    public List<String> getColumnNames()
+    {
+        return cqlTable.columns().stream()
+                       .map(CqlField::name)
+                       .collect(Collectors.toList());
+    }
+
+    /**
+     * Removes table options {@code read_repair_chance} and {@code dclocal_read_repair_chance}
+     * that were deprecated in Cassandra 4.0 from the table creation CQL statement
+     *
+     * @param cql table creation CQL statement
+     * @return table creation CQL statement with deprecated table options removed
+     */
+    @VisibleForTesting
+    @NotNull
+    static String removeDeprecatedOptions(@NotNull String cql)
+    {
+        cql = DEPRECATED_OPTIONS.matcher(cql).replaceAll("");
+        cql = LEADING_CONJUNCTION.matcher(cql).replaceAll("");
+        return cql;
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/DataTransferApi.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/DataTransferApi.java
new file mode 100644
index 0000000..c0786a2
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/DataTransferApi.java
@@ -0,0 +1,63 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+import java.io.Serializable;
+import java.nio.file.Path;
+import java.util.List;
+
+import org.apache.cassandra.spark.common.MD5Hash;
+import org.apache.cassandra.spark.common.client.ClientException;
+import org.apache.cassandra.spark.common.model.CassandraInstance;
+import org.jetbrains.annotations.Nullable;
+
+public interface DataTransferApi extends Serializable
+{
+    class RemoteCommitResult
+    {
+        public final boolean isSuccess;
+        public final List<String> failedUuids;
+        public final List<String> successfulUuids;
+        public final String stdErr;
+
+        public RemoteCommitResult(boolean isSuccess,
+                                  @Nullable List<String> failedUuids,
+                                  List<String> successfulUuids,
+                                  @Nullable String stdErr)
+        {
+            this.isSuccess = isSuccess;
+            this.failedUuids = failedUuids;
+            this.successfulUuids = successfulUuids;
+            this.stdErr = stdErr;
+        }
+    }
+
+    RemoteCommitResult commitSSTables(CassandraInstance instance,
+                                      String migrationId,
+                                      List<String> uuids) throws ClientException;
+
+    void cleanUploadSession(CassandraInstance instance, String sessionID, String jobID) throws ClientException;
+
+    void uploadSSTableComponent(Path componentFile,
+                                int ssTableIdx,
+                                CassandraInstance instance,
+                                String sessionID,
+                                MD5Hash fileHash) throws ClientException;
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/DecoratedKey.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/DecoratedKey.java
new file mode 100644
index 0000000..8d6e903
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/DecoratedKey.java
@@ -0,0 +1,92 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+import java.io.Serializable;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.spark.bulkwriter.util.FastByteOperations;
+import org.apache.cassandra.spark.utils.ByteBufferUtils;
+import org.jetbrains.annotations.NotNull;
+
+public class DecoratedKey implements Comparable<DecoratedKey>, Serializable
+{
+    @NotNull private final BigInteger token;
+    @NotNull private final ByteBuffer key;
+
+    DecoratedKey(@NotNull BigInteger token, @NotNull ByteBuffer key)
+    {
+        this.token = token;
+        this.key = key;
+    }
+
+    @NotNull
+    public BigInteger getToken()
+    {
+        return token;
+    }
+
+    @NotNull
+    public ByteBuffer getKey()
+    {
+        return key;
+    }
+
+    @Override
+    public int compareTo(@NotNull DecoratedKey that)
+    {
+        int cmp = token.compareTo(that.token);
+        if (cmp != 0)
+        {
+            return cmp;
+        }
+        return FastByteOperations.compareUnsigned(key, that.key);
+    }
+
+    @Override
+    public boolean equals(Object other)
+    {
+        if (this == other)
+        {
+            return true;
+        }
+        if (other == null || this.getClass() != other.getClass())
+        {
+            return false;
+        }
+
+        DecoratedKey that = (DecoratedKey) other;
+        return this.token.equals(that.token) && this.key.equals(that.key);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return token.hashCode() ^ key.hashCode();
+    }
+
+    @Override
+    public String toString()
+    {
+        String keystring = ByteBufferUtils.toHexString(getKey());
+        return "SBW-DecoratedKey(" + getToken() + ", " + keystring + ")";
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/InstanceAvailability.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/InstanceAvailability.java
new file mode 100644
index 0000000..2aff548
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/InstanceAvailability.java
@@ -0,0 +1,53 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+public enum InstanceAvailability
+{
+    AVAILABLE("is available"),
+    UNAVAILABLE_DOWN("is down"),
+    /**
+     * This node is listed in the cluster block list
+     */
+    UNAVAILABLE_BLOCKED("is blocked"),
+    /**
+     * This node is JOINING, and it's gossip state is BOOT_REPLACE, which means it's a host replacement.
+     * We can support SBW jobs with host replacements ongoing, but we treat them as UNAVAILABLE so the
+     * job's success or failure depends on its consistency level requirements.
+     */
+    UNAVAILABLE_REPLACEMENT("is a host replacement"),
+    /**
+     * INVALID_STATE is true when a node is in any state but NORMAL, except for host replacements (see above).
+     * The SBW doesn't currently support hosts in non-NORMAL states.
+     */
+    INVALID_STATE("is in an invalid state");
+
+    private final String message;
+
+    InstanceAvailability(String message)
+    {
+        this.message = message;
+    }
+
+    public String getMessage()
+    {
+        return message;
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/JobInfo.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/JobInfo.java
new file mode 100644
index 0000000..945012b
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/JobInfo.java
@@ -0,0 +1,54 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+import java.io.Serializable;
+import java.util.UUID;
+
+import org.apache.cassandra.spark.bulkwriter.token.ConsistencyLevel;
+
+public interface JobInfo extends Serializable
+{
+    // Job Information API - should this really just move back to Config? Here to try to reduce the violations of the Law of Demeter more than anything else
+    ConsistencyLevel getConsistencyLevel();
+
+    String getLocalDC();
+
+    RowBufferMode getRowBufferMode();
+
+    int getSstableDataSizeInMB();
+
+    int getSstableBatchSize();
+
+    int getCommitBatchSize();
+
+    int getCommitThreadsPerInstance();
+
+    UUID getId();
+
+    TokenPartitioner getTokenPartitioner();
+
+    boolean validateSSTables();
+
+    boolean skipExtendedVerify();
+
+    String getFullTableName();
+    boolean getSkipClean();
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/LoadNotSupportedException.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/LoadNotSupportedException.java
new file mode 100644
index 0000000..37cd600
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/LoadNotSupportedException.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.cassandra.spark.bulkwriter;
+
+public class LoadNotSupportedException extends UnsupportedOperationException
+{
+    public LoadNotSupportedException(String message)
+    {
+        super(message);
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/RecordWriter.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/RecordWriter.java
new file mode 100644
index 0000000..a5aaf8d
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/RecordWriter.java
@@ -0,0 +1,205 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.math.BigInteger;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.time.Duration;
+import java.time.Instant;
+import java.util.Iterator;
+import java.util.List;
+import java.util.UUID;
+import java.util.function.BiFunction;
+import java.util.function.Supplier;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Range;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.sidecar.common.data.TimeSkewResponse;
+import org.apache.spark.InterruptibleIterator;
+import org.apache.spark.TaskContext;
+import scala.Tuple2;
+
+import static org.apache.cassandra.spark.utils.ScalaConversionUtils.asScalaIterator;
+
+@SuppressWarnings({"ConstantConditions"})
+public class RecordWriter implements Serializable
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(RecordWriter.class);
+
+    private final BulkWriterContext writerContext;
+    private Supplier<TaskContext> taskContextSupplier;
+    private final BiFunction<BulkWriterContext, Path, SSTableWriter> tableWriterSupplier;
+    private SSTableWriter sstableWriter = null;
+    private int batchNumber = 0;
+    private int batchSize = 0;
+
+    public RecordWriter(BulkWriterContext writerContext)
+    {
+        this(writerContext, TaskContext::get, SSTableWriter::new);
+    }
+
+    @VisibleForTesting
+    RecordWriter(BulkWriterContext writerContext,
+                 Supplier<TaskContext> taskContextSupplier,
+                 BiFunction<BulkWriterContext, Path, SSTableWriter> tableWriterSupplier)
+    {
+        this.writerContext = writerContext;
+        this.taskContextSupplier = taskContextSupplier;
+        this.tableWriterSupplier = tableWriterSupplier;
+    }
+
+    private Range<BigInteger> getTokenRange(TaskContext taskContext)
+    {
+        return writerContext.job().getTokenPartitioner().getTokenRange(taskContext.partitionId());
+    }
+
+    private String getStreamId(TaskContext taskContext)
+    {
+        return String.format("%d-%s", taskContext.partitionId(), UUID.randomUUID());
+    }
+
+    public StreamResult write(Iterator<Tuple2<DecoratedKey, Object[]>> sourceIterator)
+    {
+        TaskContext taskContext = taskContextSupplier.get();
+        LOGGER.info("[{}]: Processing Bulk Writer partition", taskContext.partitionId());
+        scala.collection.Iterator<scala.Tuple2<DecoratedKey, Object[]>> dataIterator =
+                new InterruptibleIterator<>(taskContext, asScalaIterator(sourceIterator));
+        StreamSession streamSession = createStreamSession(taskContext);
+        validateAcceptableTimeSkewOrThrow(streamSession.replicas);
+        int partitionId = taskContext.partitionId();
+        Range<BigInteger> range = getTokenRange(taskContext);
+        JobInfo job = writerContext.job();
+        Path baseDir = Paths.get(System.getProperty("java.io.tmpdir"),
+                                 job.getId().toString(),
+                                 Integer.toString(taskContext.stageAttemptNumber()),
+                                 Integer.toString(taskContext.attemptNumber()),
+                                 Integer.toString(partitionId));
+        try
+        {
+            while (dataIterator.hasNext())
+            {
+                maybeCreateTableWriter(partitionId, baseDir);
+                writeRow(dataIterator, partitionId, range);
+                checkBatchSize(streamSession, partitionId, job);
+            }
+
+            if (batchSize != 0)
+            {
+               finalizeSSTable(streamSession, partitionId, sstableWriter, batchNumber, batchSize);
+            }
+
+            LOGGER.info("[{}] Done with all writers and waiting for stream to complete", partitionId);
+            return streamSession.close();
+        }
+        catch (Exception exception)
+        {
+            throw new RuntimeException(exception);
+        }
+    }
+
+    private void validateAcceptableTimeSkewOrThrow(List<RingInstance> replicas)
+    {
+        TimeSkewResponse timeSkewResponse = writerContext.cluster().getTimeSkew(replicas);
+        Instant localNow = Instant.now();
+        Instant remoteNow = Instant.ofEpochMilli(timeSkewResponse.currentTime);
+        Duration range = Duration.ofMinutes(timeSkewResponse.allowableSkewInMinutes);
+        if (localNow.isBefore(remoteNow.minus(range)) || localNow.isAfter(remoteNow.plus(range)))
+        {
+            String message = String.format("Time skew between Spark and Cassandra is too large. "
+                                         + "Allowable skew is %d minutes. "
+                                         + "Spark executor time is %s, Cassandra instance time is %s",
+                                           timeSkewResponse.allowableSkewInMinutes, localNow, remoteNow);
+            throw new UnsupportedOperationException(message);
+        }
+    }
+
+    public void writeRow(scala.collection.Iterator<Tuple2<DecoratedKey, Object[]>> dataIterator,
+                         int partitionId,
+                         Range<BigInteger> range) throws IOException
+    {
+        Tuple2<DecoratedKey, Object[]> tuple = dataIterator.next();
+        DecoratedKey key = tuple._1();
+        BigInteger token = key.getToken();
+        Preconditions.checkState(range.contains(token),
+                                 String.format("Received Token %s outside of expected range %s", token, range));
+        try
+        {
+            sstableWriter.addRow(token, tuple._2());
+        }
+        catch (RuntimeException exception)
+        {
+            String message = String.format("[%s]: Failed to write data to SSTable: SBW DecoratedKey was %s",
+                                           partitionId, key);
+            LOGGER.error(message, exception);
+            throw exception;
+        }
+    }
+
+    public void checkBatchSize(StreamSession streamSession, int partitionId, JobInfo job) throws IOException
+    {
+        batchSize++;
+        if (batchSize > job.getSstableBatchSize())
+        {
+            finalizeSSTable(streamSession, partitionId, sstableWriter, batchNumber, batchSize);
+
+            sstableWriter = null;
+            batchSize = 0;
+
+        }
+    }
+
+    public void maybeCreateTableWriter(int partitionId, Path baseDir) throws IOException
+    {
+        if (sstableWriter == null)
+        {
+            Path outDir = Paths.get(baseDir.toString(), Integer.toString(++batchNumber));
+            Files.createDirectories(outDir);
+
+            sstableWriter = tableWriterSupplier.apply(writerContext, outDir);
+
+            LOGGER.info("[{}][{}] Created new SSTable writer", partitionId, batchNumber);
+        }
+    }
+
+    private void finalizeSSTable(StreamSession streamSession,
+                                 int partitionId,
+                                 SSTableWriter sstableWriter,
+                                 int batchNumber,
+                                 int batchSize) throws IOException
+    {
+        LOGGER.info("[{}][{}] Closing writer and scheduling SStable stream with {} rows",
+                    partitionId, batchNumber, batchSize);
+        sstableWriter.close(writerContext, partitionId);
+        streamSession.scheduleStream(sstableWriter);
+    }
+
+    private StreamSession createStreamSession(TaskContext taskContext)
+    {
+        return new StreamSession(writerContext, getStreamId(taskContext), getTokenRange(taskContext));
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/RingInstance.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/RingInstance.java
new file mode 100644
index 0000000..75a2896
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/RingInstance.java
@@ -0,0 +1,144 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.Serializable;
+import java.util.Objects;
+
+import org.apache.cassandra.sidecar.common.data.RingEntry;
+import org.apache.cassandra.spark.common.model.CassandraInstance;
+import org.jetbrains.annotations.Nullable;
+
+public class RingInstance implements CassandraInstance, Serializable
+{
+    private static final long serialVersionUID = 4399143234683369652L;
+    private RingEntry ringEntry;
+
+    public RingInstance(RingEntry ringEntry)
+    {
+        this.ringEntry = ringEntry;
+    }
+
+    @Override
+    public String getToken()
+    {
+        return ringEntry.token();
+    }
+
+    @Override
+    public String getNodeName()
+    {
+        return ringEntry.fqdn();
+    }
+
+    @Override
+    public String getDataCenter()
+    {
+        return ringEntry.datacenter();
+    }
+
+    /**
+     * Custom equality that compares the token, fully qualified domain name, the port, and the datacenter
+     *
+     * @param other the other instance
+     * @return true if both instances are equal, false otherwise
+     */
+    @Override
+    public boolean equals(@Nullable Object other)
+    {
+        if (other == null || !(other instanceof RingInstance))
+        {
+            return false;
+        }
+
+        RingInstance that = (RingInstance) other;
+        return Objects.equals(this.ringEntry.token(), that.ringEntry.token())
+            && Objects.equals(this.ringEntry.fqdn(), that.ringEntry.fqdn())
+            && this.ringEntry.port() == that.ringEntry.port()
+            && Objects.equals(this.ringEntry.datacenter(), that.ringEntry.datacenter());
+    }
+
+    /**
+     * Custom hashCode that compares the token, fully qualified domain name, the port, and the datacenter
+     *
+     * @return The hashcode of this instance based on the important fields
+     */
+    @Override
+    public int hashCode()
+    {
+        return Objects.hash(ringEntry.token(), ringEntry.fqdn(), ringEntry.port(), ringEntry.datacenter());
+    }
+
+    @Override
+    public String toString()
+    {
+        return ringEntry.toString();
+    }
+
+    public RingEntry getRingInstance()
+    {
+        return ringEntry;
+    }
+
+    private void writeObject(ObjectOutputStream out) throws IOException
+    {
+        out.writeUTF(ringEntry.address());
+        out.writeInt(ringEntry.port());
+        out.writeUTF(ringEntry.datacenter());
+        out.writeUTF(ringEntry.load());
+        out.writeUTF(ringEntry.token());
+        out.writeUTF(ringEntry.fqdn());
+        out.writeUTF(ringEntry.rack());
+        out.writeUTF(ringEntry.hostId());
+        out.writeUTF(ringEntry.status());
+        out.writeUTF(ringEntry.state());
+        out.writeUTF(ringEntry.owns());
+    }
+
+    private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException
+    {
+        String address = in.readUTF();
+        int port = in.readInt();
+        String datacenter = in.readUTF();
+        String load = in.readUTF();
+        String token = in.readUTF();
+        String fqdn = in.readUTF();
+        String rack = in.readUTF();
+        String hostId = in.readUTF();
+        String status = in.readUTF();
+        String state = in.readUTF();
+        String owns = in.readUTF();
+        ringEntry = new RingEntry.Builder().datacenter(datacenter)
+                                           .address(address)
+                                           .port(port)
+                                           .rack(rack)
+                                           .status(status)
+                                           .state(state)
+                                           .load(load)
+                                           .owns(owns)
+                                           .token(token)
+                                           .fqdn(fqdn)
+                                           .hostId(hostId)
+                                           .build();
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/RowBufferMode.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/RowBufferMode.java
new file mode 100644
index 0000000..d54ce3f
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/RowBufferMode.java
@@ -0,0 +1,26 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+public enum RowBufferMode
+{
+    BUFFERRED,
+    UNBUFFERRED
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/SSTableWriter.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/SSTableWriter.java
new file mode 100644
index 0000000..bf982ef
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/SSTableWriter.java
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.cassandra.spark.bulkwriter;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.math.BigInteger;
+import java.nio.file.DirectoryStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.security.MessageDigest;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Range;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.bridge.CassandraBridgeFactory;
+import org.apache.cassandra.bridge.CassandraVersion;
+import org.apache.cassandra.bridge.CassandraVersionFeatures;
+import org.apache.cassandra.spark.common.MD5Hash;
+import org.apache.cassandra.spark.common.SSTables;
+import org.apache.cassandra.spark.data.DataLayer;
+import org.apache.cassandra.spark.data.LocalDataLayer;
+import org.apache.cassandra.spark.reader.StreamScanner;
+import org.jetbrains.annotations.NotNull;
+
+@SuppressWarnings("WeakerAccess")
+public class SSTableWriter
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(SSTableWriter.class);
+
+    public static final String CASSANDRA_VERSION_PREFIX = "cassandra-";
+
+    private final Path outDir;
+    private final org.apache.cassandra.bridge.SSTableWriter cqlSSTableWriter;
+    private BigInteger minToken = null;
+    private BigInteger maxToken = null;
+    private final Map<Path, MD5Hash> fileHashes = new HashMap<>();
+
+    public SSTableWriter(org.apache.cassandra.bridge.SSTableWriter tableWriter, Path outDir)
+    {
+        cqlSSTableWriter = tableWriter;
+        this.outDir = outDir;
+    }
+
+    public SSTableWriter(BulkWriterContext writerContext, Path outDir)
+    {
+        this.outDir = outDir;
+
+        String lowestCassandraVersion = writerContext.cluster().getLowestCassandraVersion();
+        String packageVersion = getPackageVersion(lowestCassandraVersion);
+        LOGGER.info("Running with version " + packageVersion);
+
+        TableSchema tableSchema = writerContext.schema().getTableSchema();
+        boolean sorted = writerContext.job().getRowBufferMode() == RowBufferMode.UNBUFFERRED;
+        this.cqlSSTableWriter = SSTableWriterFactory.getSSTableWriter(
+                CassandraVersionFeatures.cassandraVersionFeaturesFromCassandraVersion(packageVersion),
+                this.outDir.toString(),
+                writerContext.cluster().getPartitioner().toString(),
+                tableSchema.createStatement,
+                tableSchema.modificationStatement,
+                sorted,
+                writerContext.job().getSstableDataSizeInMB());
+    }
+
+    @NotNull
+    public String getPackageVersion(String lowestCassandraVersion)
+    {
+        return CASSANDRA_VERSION_PREFIX + lowestCassandraVersion;
+    }
+
+    public void addRow(BigInteger token, Object[] values) throws IOException
+    {
+        if (minToken == null)
+        {
+            minToken = token;
+        }
+        maxToken = token;
+
+        cqlSSTableWriter.addRow(values);
+    }
+
+    public void close(BulkWriterContext writerContext, int partitionId) throws IOException
+    {
+        cqlSSTableWriter.close();
+        for (Path dataFile : getDataFileStream())
+        {
+            // NOTE: We calculate file hashes before re-reading so that we know what we hashed
+            //       is what we validated. Then we send these along with the files and the
+            //       receiving end re-hashes the files to make sure they still match.
+            fileHashes.putAll(calculateFileHashes(dataFile));
+        }
+        validateSSTables(writerContext, partitionId);
+    }
+
+    @VisibleForTesting
+    public void validateSSTables(@NotNull BulkWriterContext writerContext, int partitionId)
+    {
+        // NOTE: If this current implementation of SS-tables' validation proves to be a performance issue,
+        //       we will need to modify LocalDataLayer to allow scanning and compaction of single data file,
+        //       and then validate all of them in parallel threads
+        try
+        {
+            CassandraVersion version = CassandraBridgeFactory.getCassandraVersion(writerContext.cluster().getLowestCassandraVersion());
+            String keyspace = writerContext.cluster().getRing(true).getKeyspace();
+            String schema = writerContext.schema().getTableSchema().createStatement;
+            String directory = getOutDir().toString();
+            DataLayer layer = new LocalDataLayer(version, keyspace, schema, directory);
+            try (StreamScanner scanner = layer.openCompactionScanner(partitionId, Collections.emptyList(), null))
+            {
+                while (scanner.hasNext())
+                {
+                    scanner.advanceToNextColumn();
+                }
+            }
+        }
+        catch (IOException exception)
+        {
+            LOGGER.error("[{}]: Unexpected exception while validating SSTables {}", partitionId, getOutDir());
+            throw new RuntimeException(exception);
+        }
+    }
+
+    private DirectoryStream<Path> getDataFileStream() throws IOException
+    {
+        return Files.newDirectoryStream(getOutDir(), "*Data.db");
+    }
+
+    private Map<Path, MD5Hash> calculateFileHashes(Path dataFile) throws IOException
+    {
+        Map<Path, MD5Hash> fileHashes = new HashMap<>();
+        try (DirectoryStream<Path> filesToHash =
+                Files.newDirectoryStream(dataFile.getParent(), SSTables.getSSTableBaseName(dataFile) + "*"))
+        {
+            for (Path path: filesToHash)
+            {
+                fileHashes.put(path, calculateFileHash(path));
+            }
+        }
+        return fileHashes;
+    }
+
+    private MD5Hash calculateFileHash(Path path) throws IOException
+    {
+        try (InputStream is = Files.newInputStream(path))
+        {
+            MessageDigest computedMd5 = DigestUtils.updateDigest(DigestUtils.getMd5Digest(), is);
+            return MD5Hash.fromDigest(computedMd5);
+        }
+    }
+
+    public Range<BigInteger> getTokenRange()
+    {
+        return Range.closed(minToken, maxToken);
+    }
+
+    public Path getOutDir()
+    {
+        return outDir;
+    }
+
+    public Map<Path, MD5Hash> getFileHashes()
+    {
+        return fileHashes;
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/SSTableWriterFactory.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/SSTableWriterFactory.java
new file mode 100644
index 0000000..832e370
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/SSTableWriterFactory.java
@@ -0,0 +1,50 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+import org.apache.cassandra.bridge.CassandraBridge;
+import org.apache.cassandra.bridge.CassandraBridgeFactory;
+import org.apache.cassandra.bridge.CassandraVersionFeatures;
+import org.apache.cassandra.bridge.SSTableWriter;
+
+public final class SSTableWriterFactory
+{
+    private SSTableWriterFactory()
+    {
+        throw new IllegalStateException(getClass() + " is static utility class and shall not be instantiated");
+    }
+
+    public static SSTableWriter getSSTableWriter(CassandraVersionFeatures serverVersion,
+                                                 String inDirectory,
+                                                 String partitioner,
+                                                 String createStatement,
+                                                 String insertStatement,
+                                                 boolean sorted,
+                                                 int bufferSizeMB)
+    {
+        CassandraBridge cassandraBridge = CassandraBridgeFactory.get(serverVersion);
+        return cassandraBridge.getSSTableWriter(inDirectory,
+                                                partitioner,
+                                                createStatement,
+                                                insertStatement,
+                                                sorted,
+                                                bufferSizeMB);
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/SchemaInfo.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/SchemaInfo.java
new file mode 100644
index 0000000..ca95618
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/SchemaInfo.java
@@ -0,0 +1,27 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+import java.io.Serializable;
+
+public interface SchemaInfo extends Serializable
+{
+    TableSchema getTableSchema();
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/SidecarDataTransferApi.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/SidecarDataTransferApi.java
new file mode 100644
index 0000000..4d5b25b
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/SidecarDataTransferApi.java
@@ -0,0 +1,153 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+import java.nio.file.Path;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.sidecar.client.SidecarClient;
+import org.apache.cassandra.sidecar.client.SidecarInstanceImpl;
+import org.apache.cassandra.sidecar.client.request.ImportSSTableRequest;
+import org.apache.cassandra.sidecar.common.data.SSTableImportResponse;
+import org.apache.cassandra.spark.common.MD5Hash;
+import org.apache.cassandra.spark.common.client.ClientException;
+import org.apache.cassandra.spark.common.model.CassandraInstance;
+
+/**
+ * A {@link DataTransferApi} implementation that interacts with Cassandra Sidecar
+ */
+public class SidecarDataTransferApi implements DataTransferApi
+{
+    private static final long serialVersionUID = 2563347232666882754L;
+    private static final Logger LOGGER = LoggerFactory.getLogger(SidecarDataTransferApi.class);
+    private static final String SSTABLE_NAME_SEPARATOR = "-";
+    private static final int SSTABLE_GENERATION_REVERSE_OFFSET = 3;
+
+    private final transient SidecarClient sidecarClient;
+    private final JobInfo job;
+    private final BulkSparkConf conf;
+
+    public SidecarDataTransferApi(SidecarClient sidecarClient, JobInfo job, BulkSparkConf conf)
+    {
+        this.sidecarClient = sidecarClient;
+        this.job = job;
+        this.conf = conf;
+    }
+
+    @Override
+    public void uploadSSTableComponent(Path componentFile,
+                                       int ssTableIdx,
+                                       CassandraInstance instance,
+                                       String sessionID,
+                                       MD5Hash fileHash) throws ClientException
+    {
+        String componentName = updateComponentName(componentFile, ssTableIdx);
+        String uploadId = getUploadId(sessionID, job.getId().toString());
+        try
+        {
+            sidecarClient.uploadSSTableRequest(toSidecarInstance(instance), conf.keyspace, conf.table, uploadId,
+                                               componentName, fileHash.toString(),
+                                               componentFile.toAbsolutePath().toString()).get();
+        }
+        catch (ExecutionException | InterruptedException exception)
+        {
+            LOGGER.warn("Failed to upload file={}, keyspace={}, table={}, uploadId={}, componentName={}, instance={}",
+                        componentFile, conf.keyspace, conf.table, uploadId, componentName, instance);
+            throw new ClientException(
+                    String.format("Failed to upload file=%s into keyspace=%s, table=%s, componentName=%s with uploadId=%s to instance=%s",
+                                  componentFile, conf.keyspace, conf.table, componentName, uploadId, instance), exception);
+        }
+    }
+
+    @Override
+    public RemoteCommitResult commitSSTables(CassandraInstance instance,
+                                             String migrationId,
+                                             List<String> uuids) throws ClientException
+    {
+        if (uuids.size() != 1)
+        {
+            throw new UnsupportedOperationException("Only a single UUID is supported, you provided " + uuids.size());
+        }
+        String uploadId = getUploadId(uuids.get(0), job.getId().toString());
+        ImportSSTableRequest.ImportOptions importOptions = new ImportSSTableRequest.ImportOptions();
+
+        if (job.validateSSTables())
+        {
+            importOptions.verifySSTables(true)
+                         .extendedVerify(!job.skipExtendedVerify());
+        }
+
+        try
+        {
+            SSTableImportResponse response =
+            sidecarClient.importSSTableRequest(toSidecarInstance(instance), conf.keyspace, conf.table, uploadId, importOptions).get();
+            if (response.success())
+            {
+                return new RemoteCommitResult(response.success(), Collections.emptyList(), Collections.singletonList(uploadId), null);
+            }
+            return new RemoteCommitResult(response.success(), Collections.singletonList(uploadId), Collections.emptyList(), null);
+        }
+        catch (ExecutionException | InterruptedException exception)
+        {
+            throw new ClientException("Failed to commit sorted string tables", exception);
+        }
+    }
+
+    @Override
+    public void cleanUploadSession(CassandraInstance instance, String sessionID, String jobID) throws ClientException
+    {
+        String uploadId = getUploadId(sessionID, jobID);
+        try
+        {
+            sidecarClient.cleanUploadSession(toSidecarInstance(instance), uploadId).get();
+        }
+        catch (ExecutionException | InterruptedException exception)
+        {
+            LOGGER.warn("Failed to clean upload uploadId={}, instance={}", uploadId, instance);
+            throw new ClientException("Failed to clean the upload session with ID " + uploadId, exception);
+        }
+    }
+
+    protected String updateComponentName(Path componentFile, int ssTableIdx)
+    {
+        String[] ssTableNameParts = componentFile.getFileName().toString().split(SSTABLE_NAME_SEPARATOR);
+        if (ssTableNameParts.length < SSTABLE_GENERATION_REVERSE_OFFSET)
+        {
+            throw new IllegalArgumentException("Invalid component file name: " + componentFile.getFileName());
+        }
+        ssTableNameParts[ssTableNameParts.length - SSTABLE_GENERATION_REVERSE_OFFSET] = Integer.toString(ssTableIdx);
+        return String.join(SSTABLE_NAME_SEPARATOR, ssTableNameParts);
+    }
+
+    protected String getUploadId(String sessionID, String jobId)
+    {
+        return sessionID + "-" + jobId;
+    }
+
+    private SidecarInstanceImpl toSidecarInstance(CassandraInstance instance)
+    {
+        return new SidecarInstanceImpl(instance.getNodeName(), conf.getSidecarPort());
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/SqlToCqlTypeConverter.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/SqlToCqlTypeConverter.java
new file mode 100644
index 0000000..03772bd
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/SqlToCqlTypeConverter.java
@@ -0,0 +1,726 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+import java.io.Serializable;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.net.InetAddress;
+import java.nio.ByteBuffer;
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.base.Preconditions;
+import com.google.common.net.InetAddresses;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.spark.data.CqlField;
+import org.apache.cassandra.spark.utils.UUIDs;
+import scala.Tuple2;
+
+import static org.apache.cassandra.spark.utils.ScalaConversionUtils.asJavaIterable;
+
+@SuppressWarnings("unchecked")
+public final class SqlToCqlTypeConverter implements Serializable
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(SqlToCqlTypeConverter.class);
+
+    public static final String ASCII = "ascii";
+    public static final String BIGINT = "bigint";
+    public static final String BLOB = "blob";
+    public static final String BOOLEAN = "boolean";
+    public static final String COUNTER = "counter";
+    public static final String CUSTOM = "custom";
+    public static final String DATE = "date";
+    public static final String DECIMAL = "decimal";
+    public static final String DOUBLE = "double";
+    public static final String FLOAT = "float";
+    public static final String FROZEN = "frozen";
+    public static final String INET = "inet";
+    public static final String INT = "int";
+    public static final String LIST = "list";
+    public static final String MAP = "map";
+    public static final String SET = "set";
+    public static final String SMALLINT = "smallint";
+    public static final String TEXT = "text";
+    public static final String TIME = "time";
+    public static final String UUID = "uuid";
+    public static final String TIMESTAMP = "timestamp";
+    public static final String TIMEUUID = "timeuuid";
+    public static final String TINYINT = "tinyint";
+    public static final String TUPLE = "tuple";
+    public static final String UDT = "udt";
+    public static final String VARCHAR = "varchar";
+    public static final String VARINT = "varint";
+
+    private static final NoOp<Object> NO_OP_CONVERTER = new NoOp<>();
+    private static final LongConverter LONG_CONVERTER = new LongConverter();
+    private static final BytesConverter BYTES_CONVERTER = new BytesConverter();
+    private static final BigDecimalConverter BIG_DECIMAL_CONVERTER = new BigDecimalConverter();
+    private static final IntegerConverter INTEGER_CONVERTER = new IntegerConverter();
+    private static final TimestampConverter TIMESTAMP_CONVERTER = new TimestampConverter();
+    private static final TimeConverter TIME_CONVERTER = new TimeConverter();
+    private static final UUIDConverter UUID_CONVERTER = new UUIDConverter();
+    private static final BigIntegerConverter BIG_INTEGER_CONVERTER = new BigIntegerConverter();
+    private static final TimeUUIDConverter TIME_UUID_CONVERTER = new TimeUUIDConverter();
+    private static final InetAddressConverter INET_ADDRESS_CONVERTER = new InetAddressConverter();
+    private static final DateConverter DATE_CONVERTER = new DateConverter();
+
+    private SqlToCqlTypeConverter()
+    {
+        throw new IllegalStateException(getClass() + " is static utility class and shall not be instantiated");
+    }
+
+    /**
+     * Method to get appropriate converter for a given CQL datatype
+     *
+     * @param cqlType the Cassandra data type from which a converter is needed
+     * @return a type converter that knows how to handle the appropriate type
+     */
+    public static Converter<?> getConverter(CqlField.CqlType cqlType)
+    {
+        String cqlName = cqlType.name().toLowerCase();
+        switch (cqlName)
+        {
+            case CUSTOM:
+                return determineCustomConvert((CqlField.CqlCustom) cqlType);
+            case ASCII:
+                return NO_OP_CONVERTER;
+            case BIGINT:
+                return LONG_CONVERTER;
+            case BLOB:
+                return BYTES_CONVERTER;
+            case BOOLEAN:
+                return NO_OP_CONVERTER;
+            case COUNTER:
+                return NO_OP_CONVERTER;
+            case DECIMAL:
+                return BIG_DECIMAL_CONVERTER;
+            case DOUBLE:
+                return NO_OP_CONVERTER;
+            case FLOAT:
+                return NO_OP_CONVERTER;
+            case FROZEN:
+                assert cqlType instanceof CqlField.CqlFrozen;
+                CqlField.CqlFrozen frozen = (CqlField.CqlFrozen) cqlType;
+                return getConverter(frozen.inner());
+            case INT:
+                return INTEGER_CONVERTER;
+            case TEXT:
+                return NO_OP_CONVERTER;
+            case TIMESTAMP:
+                return TIMESTAMP_CONVERTER;
+            case TIME:
+                return TIME_CONVERTER;
+            case UUID:
+                return UUID_CONVERTER;
+            case VARCHAR:
+                return NO_OP_CONVERTER;
+            case VARINT:
+                return BIG_INTEGER_CONVERTER;
+            case TIMEUUID:
+                return TIME_UUID_CONVERTER;
+            case INET:
+                return INET_ADDRESS_CONVERTER;
+            case DATE:
+                return DATE_CONVERTER;
+            case SMALLINT:
+                return NO_OP_CONVERTER;
+            case TINYINT:
+                return NO_OP_CONVERTER;
+            case LIST:
+                return new ListConverter<>((CqlField.CqlCollection) cqlType);
+            case MAP:
+                assert cqlType instanceof CqlField.CqlMap;
+                return new MapConverter<>((CqlField.CqlMap) cqlType);
+            case SET:
+                return new SetConverter<>((CqlField.CqlCollection) cqlType);
+            case UDT:
+                return NO_OP_CONVERTER;
+            case TUPLE:
+                return NO_OP_CONVERTER;
+            default:
+                LOGGER.warn("Unable to match type={}. Defaulting to NoOp Converter", cqlName);
+                return NO_OP_CONVERTER;
+        }
+    }
+
+    private static Converter<?> determineCustomConvert(CqlField.CqlCustom customType)
+    {
+        Preconditions.checkArgument(customType.name().equalsIgnoreCase(CUSTOM), "Non-custom types are not supported");
+        if (customType.customTypeClassName().equalsIgnoreCase("org.apache.cassandra.db.marshal.DateType"))
+        {
+            return TIMESTAMP_CONVERTER;
+        }
+        else
+        {
+            return NO_OP_CONVERTER;
+        }
+    }
+
+    abstract static class Converter<T> implements Serializable
+    {
+        public abstract T convertInternal(Object object) throws RuntimeException;
+
+        public T convert(Object object)
+        {
+            return convertInternal(object);
+        }
+    }
+
+    private abstract static class NullableConverter<T> extends Converter<T>
+    {
+        @Override
+        public T convert(Object object)
+        {
+            return object != null ? super.convert(object) : null;
+        }
+    }
+
+    static class NoOp<T> extends NullableConverter<T>
+    {
+        @Override
+        public T convertInternal(Object object)
+        {
+            return (T) object;
+        }
+
+        @Override
+        public String toString()
+        {
+            return "NoOp";
+        }
+    }
+
+    static class LongConverter extends NullableConverter<Long>
+    {
+        @Override
+        public Long convertInternal(Object object)
+        {
+            if (object instanceof Long
+                    || object instanceof Integer
+                    || object instanceof Short
+                    || object instanceof Byte)
+            {
+                return ((Number) object).longValue();
+            }
+            else
+            {
+                throw new RuntimeException("Unsupported conversion for LONG from " + object.getClass().getTypeName());
+            }
+        }
+
+        @Override
+        public String toString()
+        {
+            return "Long";
+        }
+    }
+
+    static class BytesConverter extends NullableConverter<ByteBuffer>
+    {
+        @Override
+        public ByteBuffer convertInternal(Object object)
+        {
+            if (object instanceof ByteBuffer)
+            {
+                return (ByteBuffer) object;
+            }
+            else if (object instanceof byte[])
+            {
+                return ByteBuffer.wrap((byte[]) object);
+            }
+            else
+            {
+                throw new RuntimeException("Unsupported conversion for BYTES from " + object.getClass().getTypeName());
+            }
+        }
+
+        @Override
+        public String toString()
+        {
+            return "Bytes";
+        }
+    }
+
+    static class BigDecimalConverter extends NullableConverter<BigDecimal>
+    {
+        @Override
+        public BigDecimal convertInternal(Object object)
+        {
+            if (object instanceof BigDecimal)
+            {
+                return (BigDecimal) object;
+            }
+            else if (object instanceof scala.math.BigDecimal)
+            {
+                return ((scala.math.BigDecimal) object).bigDecimal();
+            }
+            else if (object instanceof String)
+            {
+                return new BigDecimal((String) object);
+            }
+            else if (object instanceof Double)
+            {
+                return BigDecimal.valueOf((Double) object);
+            }
+            else if (object instanceof Float)
+            {
+                return BigDecimal.valueOf((Float) object);
+            }
+            else if (object instanceof Long)
+            {
+                return BigDecimal.valueOf((Long) object);
+            }
+            else if (object instanceof Integer)
+            {
+                return BigDecimal.valueOf((Integer) object);
+            }
+            else
+            {
+                throw new RuntimeException("Unsupported conversion for DECIMAL from " + object.getClass().getTypeName());
+            }
+        }
+
+        @Override
+        public String toString()
+        {
+            return "Decimal";
+        }
+    }
+
+    static class InetAddressConverter extends NullableConverter<InetAddress>
+    {
+        @Override
+        @SuppressWarnings("UnstableApiUsage")
+        public InetAddress convertInternal(Object object)
+        {
+            if (object instanceof InetAddress)
+            {
+                return (InetAddress) object;
+            }
+            else if (object instanceof String)
+            {
+                return InetAddresses.forString((String) object);
+            }
+            else
+            {
+                throw new RuntimeException("Unsupported conversion for INET from " + object.getClass().getTypeName());
+            }
+        }
+
+        @Override
+        public String toString()
+        {
+            return "Inet";
+        }
+    }
+
+    static class IntegerConverter extends NullableConverter<Integer>
+    {
+        @Override
+        public Integer convertInternal(Object object)
+        {
+            if (object instanceof Integer
+                    || object instanceof Short
+                    || object instanceof Byte)
+            {
+                return ((Number) object).intValue();
+            }
+            else
+            {
+                throw new RuntimeException("Unsupported conversion for INTEGER from " + object.getClass().getTypeName());
+            }
+        }
+
+        @Override
+        public String toString()
+        {
+            return "Integer";
+        }
+    }
+
+    static class BigIntegerConverter extends NullableConverter<BigInteger>
+    {
+        @Override
+        public BigInteger convertInternal(Object object)
+        {
+            if (object instanceof BigInteger)
+            {
+                return (BigInteger) object;
+            }
+            else if (object instanceof scala.math.BigInt)
+            {
+                return ((scala.math.BigInt) object).bigInteger();
+            }
+            else if (object instanceof String)
+            {
+                return new BigInteger((String) object);
+            }
+            else if (object instanceof Long)
+            {
+                return BigInteger.valueOf((Long) object);
+            }
+            else if (object instanceof Integer)
+            {
+                return BigInteger.valueOf((Integer) object);
+            }
+            else
+            {
+                throw new RuntimeException("Unsupported conversion for VARCHAR from " + object.getClass().getTypeName());
+            }
+        }
+
+        @Override
+        public String toString()
+        {
+            return "VarChar";
+        }
+    }
+
+    static class TimestampConverter extends NullableConverter<Date>
+    {
+        @Override
+        public Date convertInternal(Object object) throws RuntimeException
+        {
+            if (object instanceof Date)
+            {
+                return (Date) object;
+            }
+            else if (object instanceof Long)
+            {
+                return new Date((Long) object);
+            }
+            else
+            {
+                throw new RuntimeException("Unsupported conversion for DATE from " + object.getClass().getTypeName());
+            }
+        }
+
+        @Override
+        public String toString()
+        {
+            return "Timestamp";
+        }
+    }
+
+    static class DateConverter extends NullableConverter<Integer>
+    {
+        @Override
+        public String toString()
+        {
+            return "Date";
+        }
+
+        protected int fromDate(Date value)
+        {
+            long millisSinceEpoch = value.getTime();
+            return fromMillisSinceEpoch(millisSinceEpoch);
+        }
+
+        protected int fromMillisSinceEpoch(long millisSinceEpoch)
+        {
+            // NOTE: This code is lifted from org.apache.cassandra.serializers.SimpleDateSerializer#timeInMillisToDay.
+            //       Reproduced here due to the difficulties of referencing classes from specific versions of Cassandra
+            //       in the SBW.
+            int result = (int) TimeUnit.MILLISECONDS.toDays(millisSinceEpoch);
+            result -= Integer.MIN_VALUE;
+            return result;
+        }
+
+        @Override
+        public Integer convertInternal(Object object)
+        {
+            if (object instanceof Date)
+            {
+                return fromDate((Date) object);
+            }
+            else if (object instanceof Long)
+            {
+                return fromMillisSinceEpoch((Long) object);
+            }
+            else
+            {
+                throw new RuntimeException("Unsupported conversion for DATE from " + object.getClass().getTypeName());
+            }
+        }
+    }
+
+    static class TimeConverter extends NullableConverter<Long>
+    {
+        @Override
+        @SuppressWarnings("deprecation")
+        public Long convertInternal(Object object)
+        {
+            if (object instanceof Long)
+            {
+                long result = (Long) object;
+                if (result < 0 || result >= TimeUnit.DAYS.toNanos(1))
+                {
+                    throw new IllegalArgumentException("Input value out of bounds for Cassandra Time field: " + result);
+                }
+                return (long) object;
+            }
+            else if (object instanceof Timestamp)
+            {
+                // Here, we truncate the date information and just use the time
+                Timestamp timestamp = (Timestamp) object;
+                long rawTime = 0;
+                rawTime += TimeUnit.HOURS.toNanos(timestamp.getHours());
+                rawTime += TimeUnit.MINUTES.toNanos(timestamp.getMinutes());
+                rawTime += TimeUnit.SECONDS.toNanos(timestamp.getSeconds());
+                rawTime += timestamp.getNanos();
+                return rawTime;
+            }
+            else
+            {
+                throw new RuntimeException("Unsupported conversion for TIME from " + object.getClass().getTypeName());
+            }
+        }
+
+        @Override
+        public String toString()
+        {
+            return "Time";
+        }
+    }
+
+    static class UUIDConverter extends NullableConverter<UUID>
+    {
+        @Override
+        public UUID convertInternal(Object object)
+        {
+            if (object instanceof UUID)
+            {
+                return (UUID) object;
+            }
+            else if (object instanceof String)
+            {
+                return java.util.UUID.fromString((String) object);
+            }
+            else
+            {
+                throw new RuntimeException("Unsupported conversion for UUID from " + object.getClass().getTypeName());
+            }
+        }
+
+        @Override
+        public String toString()
+        {
+            return "UUID";
+        }
+    }
+
+    static class TimeUUIDConverter extends NullableConverter<UUID>
+    {
+        @Override
+        public UUID convertInternal(Object object)
+        {
+            UUID result;
+            if (object instanceof UUID)
+            {
+                result = (UUID) object;
+            }
+            else if (object instanceof String)
+            {
+                result = java.util.UUID.fromString((String) object);
+            }
+            else if (object instanceof Long)
+            {
+                result = UUIDs.startOf((Long) object);
+            }
+            else
+            {
+                throw new RuntimeException("Unsupported conversion for TIMEUUID from " + object.getClass().getTypeName());
+            }
+            if (result.version() == 1)
+            {
+                return result;
+            }
+            else
+            {
+                throw new RuntimeException("Attempted to convert a non-Timestamp UUID to a TimeUUID - UUID was " + object);
+            }
+        }
+
+        @Override
+        public String toString()
+        {
+            return "TimeUUID";
+        }
+    }
+
+    static class ListConverter<E> extends NullableConverter<List<E>>
+    {
+        private final Converter<?> innerConverter;
+
+        ListConverter(CqlField.CqlCollection cqlType)
+        {
+            innerConverter = getConverter(cqlType.type());
+        }
+
+        @Override
+        public List<E> convertInternal(Object object)
+        {
+            if (object instanceof scala.collection.Iterable)
+            {
+                return makeList(asJavaIterable((scala.collection.Iterable<?>) object));
+            }
+            else if (object instanceof Iterable)
+            {
+                return makeList((Iterable<?>) object);
+            }
+            else
+            {
+                throw new RuntimeException("Unsupported conversion for LIST from " + object.getClass().getTypeName());
+            }
+        }
+
+        private List<E> makeList(Iterable<?> iterable)
+        {
+            List<E> list = new ArrayList<>();
+            for (Object object : iterable)
+            {
+                list.add((E) innerConverter.convert(object));
+            }
+            return list;
+        }
+
+        @Override
+        public String toString()
+        {
+            return "List";
+        }
+    }
+
+    static class SetConverter<E> extends NullableConverter<Set<E>>
+    {
+        private final Converter<?> innerConverter;
+
+        SetConverter(CqlField.CqlCollection cqlType)
+        {
+            innerConverter = getConverter(cqlType.type());
+        }
+
+        @Override
+        public Set<E> convertInternal(Object object)
+        {
+            if (object instanceof scala.collection.Iterable)
+            {
+                return makeSet(asJavaIterable((scala.collection.Iterable<?>) object));
+            }
+            else if (object instanceof Iterable)
+            {
+                return makeSet((Iterable<?>) object);
+            }
+            else
+            {
+                throw new RuntimeException("Unsupported conversion for SET from " + object.getClass().getTypeName());
+            }
+        }
+
+        private Set<E> makeSet(Iterable<?> iterable)
+        {
+            Set<E> set = new HashSet<>();
+            for (Object object : iterable)
+            {
+                set.add((E) innerConverter.convert(object));
+            }
+            return set;
+        }
+
+        @Override
+        public String toString()
+        {
+            return "Set<" + innerConverter.toString() + ">";
+        }
+    }
+
+    static class MapConverter<K, V> extends NullableConverter<Map<K, V>>
+    {
+        private final Converter<?> keyConverter;
+        private final Converter<?> valConverter;
+
+        MapConverter(CqlField.CqlMap cqlType)
+        {
+            keyConverter = getConverter(cqlType.keyType());
+            valConverter = getConverter(cqlType.valueType());
+        }
+
+        @Override
+        public Map<K, V> convertInternal(Object object)
+        {
+            if (object instanceof scala.collection.Iterable)
+            {
+                return makeMap(asJavaIterable((scala.collection.Iterable<?>) object));
+            }
+            else if (object instanceof Iterable)
+            {
+                return makeMap((Iterable<?>) object);
+            }
+            else if (object instanceof Map)
+            {
+                return makeMap(((Map<K, V>) object).entrySet());
+            }
+            throw new RuntimeException("Unsupported conversion for MAP from " + object.getClass().getTypeName());
+        }
+
+        private Map<K, V> makeMap(Iterable<?> iterable)
+        {
+            Object key;
+            Object value;
+            Map<K, V> map = new HashMap<>();
+            for (Object object : iterable)
+            {
+                if (object instanceof Map.Entry)
+                {
+                    key = ((Map.Entry<K, V>) object).getKey();
+                    value = ((Map.Entry<K, V>) object).getValue();
+                }
+                else if (object instanceof scala.Tuple2)
+                {
+                    key = ((Tuple2<K, V>) object)._1();
+                    value = ((Tuple2<K, V>) object)._2();
+                }
+                else
+                {
+                    throw new RuntimeException("Unsupported conversion for key/value pair in MAP from " + object.getClass().getTypeName());
+                }
+                map.put((K) keyConverter.convert(key), (V) valConverter.convert(value));
+            }
+            return map;
+        }
+
+        @Override
+        public String toString()
+        {
+            return "Map";
+        }
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/StreamError.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/StreamError.java
new file mode 100644
index 0000000..da5383d
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/StreamError.java
@@ -0,0 +1,34 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+import java.io.Serializable;
+
+public class StreamError implements Serializable
+{
+    public final RingInstance instance;
+    public final String errMsg;
+
+    public StreamError(RingInstance instance, String errMsg)
+    {
+        this.instance = instance;
+        this.errMsg = errMsg;
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/StreamResult.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/StreamResult.java
new file mode 100644
index 0000000..ee8d02b
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/StreamResult.java
@@ -0,0 +1,59 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+import java.io.Serializable;
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.List;
+
+import com.google.common.collect.Range;
+
+public class StreamResult implements Serializable
+{
+    public final String sessionID;
+    public final Range<BigInteger> tokenRange;
+    public final ArrayList<StreamError> failures;
+    public List<CommitResult> commitResults;  // CHECKSTYLE IGNORE: Public mutable field
+    public final ArrayList<RingInstance> passed;
+
+    public StreamResult(String sessionID,
+                        Range<BigInteger> tokenRange,
+                        ArrayList<StreamError> failures,
+                        ArrayList<RingInstance> passed)
+    {
+        this.sessionID = sessionID;
+        this.tokenRange = tokenRange;
+        this.failures = failures;
+        this.passed = passed;
+    }
+
+    public void setCommitResults(List<CommitResult> commitResult)
+    {
+        this.commitResults = commitResult;
+    }
+
+    @Override
+    public String toString()
+    {
+        return String.format("StreamResult{sessionID='%s', tokenRange=%s, failures=%s, commitResults=%s, passed=%s}",
+                             sessionID, tokenRange, failures, commitResults, passed);
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/StreamSession.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/StreamSession.java
new file mode 100644
index 0000000..cd6a51d
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/StreamSession.java
@@ -0,0 +1,300 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+import java.io.File;
+import java.io.IOException;
+import java.math.BigInteger;
+import java.nio.file.DirectoryStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Range;
+import org.apache.commons.io.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.spark.bulkwriter.token.CassandraRing;
+import org.apache.cassandra.spark.bulkwriter.token.ReplicaAwareFailureHandler;
+import org.apache.cassandra.spark.common.MD5Hash;
+import org.apache.cassandra.spark.common.SSTables;
+
+public class StreamSession
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(StreamSession.class);
+    private final BulkWriterContext writerContext;
+    private final String sessionID;
+    private final Range<BigInteger> tokenRange;
+    final List<RingInstance> replicas;
+    private final ArrayList<StreamError> errors = new ArrayList<>();
+    private final ReplicaAwareFailureHandler<RingInstance> failureHandler;
+    private final AtomicInteger nextSSTableIdx = new AtomicInteger(1);
+    private final ExecutorService executor;
+    private final List<Future<?>> futures = new ArrayList<>();
+    private final CassandraRing<RingInstance> ring;
+    private static final String WRITE_PHASE = "UploadAndCommit";
+
+    public StreamSession(BulkWriterContext writerContext, String sessionID, Range<BigInteger> tokenRange)
+    {
+        this(writerContext, sessionID, tokenRange, Executors.newSingleThreadExecutor());
+    }
+
+    @VisibleForTesting
+    public StreamSession(BulkWriterContext writerContext,
+                         String sessionID,
+                         Range<BigInteger> tokenRange,
+                         ExecutorService executor)
+    {
+        this.writerContext = writerContext;
+        this.ring = writerContext.cluster().getRing(true);
+        this.failureHandler = new ReplicaAwareFailureHandler<>(ring);
+        this.sessionID = sessionID;
+        this.tokenRange = tokenRange;
+        this.replicas = getReplicas();
+        this.executor = executor;
+    }
+
+    public void scheduleStream(SSTableWriter ssTableWriter)
+    {
+        Preconditions.checkState(!ssTableWriter.getTokenRange().isEmpty(), "Trying to stream empty SSTable");
+
+        Preconditions.checkState(tokenRange.encloses(ssTableWriter.getTokenRange()),
+                                 String.format("SSTable range %s should be enclosed in the partition range %s",
+                                               ssTableWriter.getTokenRange(), tokenRange));
+
+        futures.add(executor.submit(() -> sendSSTables(writerContext, ssTableWriter)));
+    }
+
+    public StreamResult close() throws ExecutionException, InterruptedException
+    {
+        for (Future future : futures)
+        {
+            try
+            {
+                future.get();
+            }
+            catch (Exception exception)
+            {
+                LOGGER.error("Unexpected stream errMsg. "
+                           + "Stream errors should have converted to StreamError and sent to driver", exception);
+                throw new RuntimeException(exception);
+            }
+        }
+
+        executor.shutdown();
+        LOGGER.info("[{}]: Closing stream session. Sent {} SSTables", sessionID, futures.size());
+
+        if (futures.isEmpty())
+        {
+            return new StreamResult(sessionID, tokenRange, new ArrayList<>(), new ArrayList<>());
+        }
+        else
+        {
+            StreamResult streamResult = new StreamResult(sessionID, tokenRange, errors, new ArrayList<>(replicas));
+            List<CommitResult> cr = commit(streamResult);
+            streamResult.setCommitResults(cr);
+            LOGGER.debug("StreamResult: {}", streamResult);
+            BulkWriteValidator.validateClOrFail(failureHandler, LOGGER, WRITE_PHASE, writerContext.job());
+            return streamResult;
+        }
+    }
+
+    private List<CommitResult> commit(StreamResult streamResult) throws ExecutionException, InterruptedException
+    {
+        try (CommitCoordinator cc = CommitCoordinator.commit(writerContext, new StreamResult[]{streamResult}))
+        {
+            List<CommitResult> commitResults = cc.get();
+            LOGGER.debug("All CommitResults: {}", commitResults);
+            commitResults.forEach(cr -> BulkWriteValidator.updateFailureHandler(cr, WRITE_PHASE, failureHandler));
+            return commitResults;
+        }
+    }
+
+    @VisibleForTesting
+    List<RingInstance> getReplicas()
+    {
+        Map<Range<BigInteger>, List<RingInstance>> overlappingRanges = ring.getSubRanges(tokenRange).asMapOfRanges();
+
+        Preconditions.checkState(overlappingRanges.keySet().size() == 1,
+                                 String.format("Partition range %s is mapping more than one range %s",
+                                               tokenRange, overlappingRanges));
+
+        List<RingInstance> replicaList = overlappingRanges.values().stream()
+                                                          .flatMap(Collection::stream)
+                                                          .distinct()
+                                                          .collect(Collectors.toList());
+        List<RingInstance> availableReplicas = validateReplicas(replicaList);
+        // In order to better utilize replicas, shuffle the replicaList so each session starts writing to a different replica first
+        Collections.shuffle(availableReplicas);
+        return availableReplicas;
+    }
+
+    private List<RingInstance> validateReplicas(List<RingInstance> replicaList)
+    {
+        Map<Boolean, List<RingInstance>> groups = replicaList.stream()
+                .collect(Collectors.partitioningBy(writerContext.cluster()::instanceIsAvailable));
+        groups.get(false).forEach(instance -> {
+            String errorMessage = String.format("Instance %s is not available.", instance.getNodeName());
+            failureHandler.addFailure(tokenRange, instance, errorMessage);
+            errors.add(new StreamError(instance, errorMessage));
+        });
+        return groups.get(true);
+    }
+
+    private void sendSSTables(BulkWriterContext writerContext, SSTableWriter ssTableWriter)
+    {
+        try (DirectoryStream<Path> dataFileStream = Files.newDirectoryStream(ssTableWriter.getOutDir(), "*Data.db"))
+        {
+            for (Path dataFile : dataFileStream)
+            {
+                int ssTableIdx = nextSSTableIdx.getAndIncrement();
+
+                LOGGER.info("[{}]: Pushing SSTable {} to replicas {}",
+                            sessionID, dataFile, replicas.stream()
+                                                         .map(RingInstance::getNodeName)
+                                                         .collect(Collectors.joining(",")));
+                replicas.removeIf(replica -> !trySendSSTableToReplica(writerContext, ssTableWriter, dataFile, ssTableIdx, replica));
+            }
+        }
+        catch (IOException exception)
+        {
+            LOGGER.error("[{}]: Unexpected exception while streaming SSTables {}",
+                         sessionID, ssTableWriter.getOutDir());
+            cleanAllReplicas();
+            throw new RuntimeException(exception);
+        }
+        finally
+        {
+            // Clean up SSTable files once the task is complete
+            File tempDir = ssTableWriter.getOutDir().toFile();
+            LOGGER.info("[{}]:Removing temporary files after stream session from {}", sessionID, tempDir);
+            try
+            {
+                FileUtils.deleteDirectory(tempDir);
+            }
+            catch (IOException exception)
+            {
+                LOGGER.warn("[{}]:Failed to delete temporary directory {}", sessionID, tempDir, exception);
+            }
+        }
+    }
+
+    private boolean trySendSSTableToReplica(BulkWriterContext writerContext,
+                                            SSTableWriter ssTableWriter,
+                                            Path dataFile,
+                                            int ssTableIdx,
+                                            RingInstance replica)
+    {
+        try
+        {
+            sendSSTableToReplica(writerContext, dataFile, ssTableIdx, replica, ssTableWriter.getFileHashes());
+            return true;
+        }
+        catch (Exception exception)
+        {
+            LOGGER.error("[{}]: Failed to stream range {} to instance {}",
+                         sessionID, tokenRange, replica.getNodeName(), exception);
+            writerContext.cluster().refreshClusterInfo();
+            failureHandler.addFailure(tokenRange, replica, exception.getMessage());
+            errors.add(new StreamError(replica, exception.getMessage()));
+            clean(writerContext, replica, sessionID);
+            return false;
+        }
+    }
+
+    /**
+     * Get all replicas and clean temporary state on them
+     */
+    private void cleanAllReplicas()
+    {
+        Set<RingInstance> instances = new HashSet<>(replicas);
+        errors.forEach(streamError -> instances.add(streamError.instance));
+        instances.forEach(instance -> clean(writerContext, instance, sessionID));
+    }
+
+    private void sendSSTableToReplica(BulkWriterContext writerContext,
+                                      Path dataFile,
+                                      int ssTableIdx,
+                                      RingInstance instance,
+                                      Map<Path, MD5Hash> fileHashes) throws Exception
+    {
+        try (DirectoryStream<Path> componentFileStream =
+                Files.newDirectoryStream(dataFile.getParent(), SSTables.getSSTableBaseName(dataFile) + "*"))
+        {
+            for (Path componentFile : componentFileStream)
+            {
+                if (componentFile.getFileName().toString().endsWith("Data.db"))
+                {
+                    continue;
+                }
+                sendSSTableComponent(writerContext, componentFile, ssTableIdx, instance, fileHashes.get(componentFile));
+            }
+            sendSSTableComponent(writerContext, dataFile, ssTableIdx, instance, fileHashes.get(dataFile));
+        }
+    }
+
+    private void sendSSTableComponent(BulkWriterContext writerContext,
+                                      Path componentFile,
+                                      int ssTableIdx,
+                                      RingInstance instance,
+                                      MD5Hash fileHash) throws Exception
+    {
+        Preconditions.checkNotNull(fileHash, "All files must have a hash. SSTableWriter should have calculated these. This is a bug.");
+        long fileSize = Files.size(componentFile);
+        LOGGER.info("[{}]: Uploading {} to {}: Size is {}", sessionID, componentFile, instance.getNodeName(), fileSize);
+        writerContext.transfer().uploadSSTableComponent(componentFile, ssTableIdx, instance, sessionID, fileHash);
+    }
+
+    public static void clean(BulkWriterContext writerContext, RingInstance instance, String sessionID)
+    {
+        if (writerContext.job().getSkipClean())
+        {
+            LOGGER.info("Skip clean requested - not cleaning SSTable session {} on instance {}",
+                        sessionID, instance.getNodeName());
+            return;
+        }
+        String jobID = writerContext.job().getId().toString();
+        LOGGER.info("Cleaning SSTable session {} on instance {}", sessionID, instance.getNodeName());
+        try
+        {
+            writerContext.transfer().cleanUploadSession(instance, sessionID, jobID);
+        }
+        catch (Exception exception)
+        {
+            LOGGER.warn("Failed to clean SSTables on {} for session {} and ignoring errMsg",
+                        instance.getNodeName(), sessionID, exception);
+        }
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/TableInfoProvider.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/TableInfoProvider.java
new file mode 100644
index 0000000..aaf1ff5
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/TableInfoProvider.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.cassandra.spark.bulkwriter;
+
+import java.util.List;
+
+import org.apache.cassandra.spark.common.schema.ColumnType;
+import org.apache.cassandra.spark.data.CqlField;
+
+public interface TableInfoProvider
+{
+    CqlField.CqlType getColumnType(String columnName);
+
+    List<ColumnType<?>> getPartitionKeyTypes();
+
+    boolean columnExists(String columnName);
+
+    List<String> getPartitionKeyColumnNames();
+
+    String getCreateStatement();
+
+    List<String> getPrimaryKeyColumnNames();
+
+    String getName();
+
+    String getKeyspaceName();
+
+    boolean hasSecondaryIndex();
+
+    List<String> getColumnNames();
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/TableSchema.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/TableSchema.java
new file mode 100644
index 0000000..6e5b0ae
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/TableSchema.java
@@ -0,0 +1,234 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.spark.common.schema.ColumnType;
+import org.apache.spark.sql.types.StructType;
+
+public class TableSchema implements Serializable
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(TableSchema.class);
+
+    final String createStatement;
+    final String modificationStatement;
+    final List<String> partitionKeyColumns;
+    final List<ColumnType<?>> partitionKeyColumnTypes;
+    final List<SqlToCqlTypeConverter.Converter<?>> converters;
+    private final List<Integer> keyFieldPositions;
+    private final WriteMode writeMode;
+
+    public TableSchema(StructType dfSchema, TableInfoProvider tableInfo, WriteMode writeMode)
+    {
+        this.writeMode = writeMode;
+
+        validateDataFrameCompatibility(dfSchema, tableInfo);
+        validateNoSecondaryIndexes(tableInfo);
+
+        this.createStatement = getCreateStatement(tableInfo);
+        this.modificationStatement = getModificationStatement(dfSchema, tableInfo);
+        this.partitionKeyColumns = getPartitionKeyColumnNames(tableInfo);
+        this.partitionKeyColumnTypes = getPartitionKeyColumnTypes(tableInfo);
+        this.converters = getConverters(dfSchema, tableInfo);
+        LOGGER.info("Converters: {}", converters);
+        this.keyFieldPositions = getKeyFieldPositions(dfSchema, tableInfo.getColumnNames(), getRequiredKeyColumns(tableInfo));
+    }
+
+    private List<String> getRequiredKeyColumns(TableInfoProvider tableInfo)
+    {
+        switch (writeMode)
+        {
+            case INSERT:
+                // Inserts require all primary key columns
+                return tableInfo.getPrimaryKeyColumnNames();
+            case DELETE_PARTITION:
+                // To delete a partition, we only need the partition key columns, not all primary key columns
+                return tableInfo.getPartitionKeyColumnNames();
+            default:
+                throw new UnsupportedOperationException("Unknown WriteMode provided");
+        }
+    }
+
+    public Object[] normalize(Object[] row)
+    {
+        for (int index = 0; index < row.length; index++)
+        {
+            row[index] = converters.get(index).convert(row[index]);
+        }
+        return row;
+    }
+
+    public Object[] getKeyColumns(Object[] allColumns)
+    {
+        Object[] result = new Object[keyFieldPositions.size()];
+        for (int keyFieldPosition = 0; keyFieldPosition < keyFieldPositions.size(); keyFieldPosition++)
+        {
+            Object colVal = allColumns[keyFieldPositions.get(keyFieldPosition)];
+            Preconditions.checkNotNull(colVal, "Found a null primary or composite key column in source data. All key columns must be non-null.");
+            result[keyFieldPosition] = colVal;
+        }
+        return result;
+    }
+
+    private static List<SqlToCqlTypeConverter.Converter<?>> getConverters(StructType dfSchema,
+                                                                          TableInfoProvider tableInfo)
+    {
+        return Arrays.stream(dfSchema.fieldNames())
+                     .map(tableInfo::getColumnType)
+                     .map(SqlToCqlTypeConverter::getConverter)
+                     .collect(Collectors.toList());
+    }
+
+    private static List<ColumnType<?>> getPartitionKeyColumnTypes(TableInfoProvider tableInfo)
+    {
+        return tableInfo.getPartitionKeyTypes();
+    }
+
+    private static List<String> getPartitionKeyColumnNames(TableInfoProvider tableInfo)
+    {
+        return tableInfo.getPartitionKeyColumnNames();
+    }
+
+    private static String getCreateStatement(TableInfoProvider tableInfo)
+    {
+        String createStatement = tableInfo.getCreateStatement();
+        LOGGER.info("CQL create statement for the table {}", createStatement);
+        return createStatement;
+    }
+
+    private String getModificationStatement(StructType dfSchema, TableInfoProvider tableInfo)
+    {
+        switch (writeMode)
+        {
+            case INSERT:
+                return getInsertStatement(dfSchema, tableInfo);
+            case DELETE_PARTITION:
+                return getDeleteStatement(dfSchema, tableInfo);
+            default:
+                throw new UnsupportedOperationException("Unknown WriteMode provided");
+        }
+    }
+
+    private static String getInsertStatement(StructType dfSchema, TableInfoProvider tableInfo)
+    {
+        String insertStatement = String.format("INSERT INTO %s.%s (%s) VALUES (%s);",
+                                               tableInfo.getKeyspaceName(),
+                                               tableInfo.getName(),
+                                               String.join(",", dfSchema.fieldNames()),
+                                               Arrays.stream(dfSchema.fieldNames())
+                                                     .map(field -> "?")
+                                                     .collect(Collectors.joining(",")));
+
+        LOGGER.info("CQL insert statement for the RDD {}", insertStatement);
+        return insertStatement;
+    }
+
+    private String getDeleteStatement(StructType dfSchema, TableInfoProvider tableInfo)
+    {
+        Stream<String> fieldEqualityStatements = Arrays.stream(dfSchema.fieldNames()).map(key -> key + "=?");
+        String deleteStatement = String.format("DELETE FROM %s.%s where %s;",
+                                               tableInfo.getKeyspaceName(),
+                                               tableInfo.getName(),
+                                               fieldEqualityStatements.collect(Collectors.joining(" AND ")));
+
+        LOGGER.info("CQL delete statement for the RDD {}", deleteStatement);
+        return deleteStatement;
+    }
+
+    private void validateDataFrameCompatibility(StructType dfSchema, TableInfoProvider tableInfo)
+    {
+        Set<String> dfFields = new LinkedHashSet<>();
+        Collections.addAll(dfFields, dfSchema.fieldNames());
+
+        validatePrimaryKeyColumnsProvided(tableInfo, dfFields);
+
+        switch (writeMode)
+        {
+            case INSERT:
+                validateDataframeFieldsInTable(tableInfo, dfFields);
+                return;
+            case DELETE_PARTITION:
+                validateOnlyPartitionKeyColumnsInDataframe(tableInfo, dfFields);
+                return;
+            default:
+                LOGGER.warn("Unrecognized write mode {}", writeMode);
+        }
+    }
+
+    private void validateOnlyPartitionKeyColumnsInDataframe(TableInfoProvider tableInfo, Set<String> dfFields)
+    {
+        Set<String> requiredKeyColumns = new LinkedHashSet<>(getRequiredKeyColumns(tableInfo));
+        Preconditions.checkArgument(requiredKeyColumns.equals(dfFields),
+                                    String.format("Only partition key columns (%s) are supported in the input Dataframe"
+                                                + " when WRITE_MODE=DELETE_PARTITION but (%s) columns were provided",
+                                                  String.join(",", requiredKeyColumns), String.join(",", dfFields)));
+    }
+
+    private void validatePrimaryKeyColumnsProvided(TableInfoProvider tableInfo, Set<String> dfFields)
+    {
+        // Make sure all primary key columns are provided
+        List<String> requiredKeyColumns = getRequiredKeyColumns(tableInfo);
+        Preconditions.checkArgument(dfFields.containsAll(requiredKeyColumns),
+                                    "Missing some required key components in DataFrame => " + requiredKeyColumns
+                                            .stream()
+                                            .filter(column -> !dfFields.contains(column))
+                                            .collect(Collectors.joining(",")));
+    }
+
+    private static void validateDataframeFieldsInTable(TableInfoProvider tableInfo, Set<String> dfFields)
+    {
+        // Make sure all fields in DF schema are part of table
+        String unknownFields = dfFields.stream()
+                                       .filter(columnName -> !tableInfo.columnExists(columnName))
+                                       .collect(Collectors.joining(","));
+        Preconditions.checkArgument(unknownFields.isEmpty(), "Unknown fields in data frame => " + unknownFields);
+    }
+
+    private static void validateNoSecondaryIndexes(TableInfoProvider tableInfo)
+    {
+        if (tableInfo.hasSecondaryIndex())
+        {
+            throw new RuntimeException("Bulkwriter doesn't support secondary indexes");
+        }
+    }
+
+    private static List<Integer> getKeyFieldPositions(StructType dfSchema,
+                                                      List<String> columnNames,
+                                                      List<String> keyFieldNames)
+    {
+        List<String> dfFieldNames = Arrays.asList(dfSchema.fieldNames());
+        return columnNames.stream()
+                          .filter(keyFieldNames::contains)
+                          .map(dfFieldNames::indexOf)
+                          .collect(Collectors.toList());
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/TokenPartitioner.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/TokenPartitioner.java
new file mode 100644
index 0000000..06d51f1
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/TokenPartitioner.java
@@ -0,0 +1,243 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.math.BigInteger;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.RangeSet;
+import com.google.common.collect.TreeRangeMap;
+import com.google.common.collect.TreeRangeSet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.spark.bulkwriter.token.CassandraRing;
+import org.apache.cassandra.spark.bulkwriter.token.RangeUtils;
+import org.apache.spark.Partitioner;
+
+public class TokenPartitioner extends Partitioner
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(TokenPartitioner.class);
+
+    private transient int nrPartitions;
+    private transient RangeMap<BigInteger, Integer> partitionMap;
+    private transient Map<Integer, Range<BigInteger>> reversePartitionMap;
+    private final CassandraRing<RingInstance> ring;
+    private final Integer numberSplits;
+
+    public TokenPartitioner(CassandraRing<RingInstance> ring,
+                            Integer numberSplits,
+                            int defaultParallelism,
+                            Integer cores)
+    {
+        this(ring, numberSplits, defaultParallelism, cores, true);
+    }
+
+    @VisibleForTesting
+    public TokenPartitioner(CassandraRing<RingInstance> ring,
+                            Integer numberSplits,
+                            int defaultParallelism,
+                            Integer cores,
+                            boolean randomize)
+    {
+        this.ring = ring;
+        this.numberSplits = calculateSplits(ring, numberSplits, defaultParallelism, cores);
+        setupTokenRangeMap(randomize);
+        validate();  // Intentionally not keeping this in readObject(), it is enough to validate in constructor alone
+        LOGGER.info("Partition map " + partitionMap);
+        LOGGER.info("Reverse partition map " + reversePartitionMap);
+        LOGGER.info("Number of partitions {}", nrPartitions);
+    }
+
+    @Override
+    public int numPartitions()
+    {
+        return nrPartitions;
+    }
+
+    @SuppressWarnings("ConstantConditions")
+    @Override
+    public int getPartition(Object key)
+    {
+        DecoratedKey decoratedKey = (DecoratedKey) key;
+
+        return partitionMap.get(decoratedKey.getToken());
+    }
+
+    public int numSplits()
+    {
+        return numberSplits;
+    }
+
+    public Range<BigInteger> getTokenRange(int partitionId)
+    {
+        return reversePartitionMap.get(partitionId);
+    }
+
+    private void setupTokenRangeMap(boolean randomize)
+    {
+        partitionMap = TreeRangeMap.create();
+        reversePartitionMap = new HashMap<>();
+
+        AtomicInteger nextPartitionId = new AtomicInteger(0);
+        List<Range<BigInteger>> subRanges = ring.getRangeMap().asMapOfRanges().keySet().stream()
+                .flatMap(tr -> RangeUtils.split(tr, numberSplits).stream())
+                .collect(Collectors.toList());
+        if (randomize)
+        {
+            // In order to help distribute the upload load more evenly, shuffle the subranges before assigning a partition
+            Collections.shuffle(subRanges);
+        }
+        subRanges.forEach(tr -> {
+            int partitionId = nextPartitionId.getAndIncrement();
+
+            partitionMap.put(tr, partitionId);
+            reversePartitionMap.put(partitionId, tr);
+        });
+
+        this.nrPartitions = nextPartitionId.get();
+    }
+
+    private void validate()
+    {
+        validateMapSizes();
+        validateCompleteRangeCoverage();
+        validateRangesDoNotOverlap();
+    }
+
+    private void validateRangesDoNotOverlap()
+    {
+        List<Range<BigInteger>> sortedRanges = partitionMap.asMapOfRanges().keySet().stream()
+                .sorted(Comparator.comparing(Range::lowerEndpoint))
+                .collect(Collectors.toList());
+        Range<BigInteger> previous = null;
+        for (Range<BigInteger> current : sortedRanges)
+        {
+            if (previous != null)
+            {
+                Preconditions.checkState(!current.isConnected(previous) || current.intersection(previous).isEmpty(),
+                        String.format("Two ranges in partition map are overlapping %s %s", previous, current));
+            }
+
+            previous = current;
+        }
+    }
+
+    private void validateCompleteRangeCoverage()
+    {
+        RangeSet<BigInteger> missingRangeSet = TreeRangeSet.create();
+        missingRangeSet.add(Range.closed(ring.getPartitioner().minToken(),
+                ring.getPartitioner().maxToken()));
+
+        partitionMap.asMapOfRanges().keySet().forEach(missingRangeSet::remove);
+
+        List<Range<BigInteger>> missingRanges = missingRangeSet.asRanges().stream()
+                                                                          .filter(Range::isEmpty)
+                                                                          .collect(Collectors.toList());
+        // noinspection unchecked
+        Preconditions.checkState(missingRanges.isEmpty(),
+                "There should be no missing ranges, but found " + missingRanges.toString());
+    }
+
+    private void validateMapSizes()
+    {
+        Preconditions.checkState(nrPartitions == partitionMap.asMapOfRanges().keySet().size(),
+                                 String.format("Number of partitions %d not matching with partition map size %d",
+                                               nrPartitions, partitionMap.asMapOfRanges().keySet().size()));
+        Preconditions.checkState(nrPartitions == reversePartitionMap.keySet().size(),
+                                 String.format("Number of partitions %d not matching with reverse partition map size %d",
+                                               nrPartitions, reversePartitionMap.keySet().size()));
+        Preconditions.checkState(nrPartitions >= ring.getRangeMap().asMapOfRanges().keySet().size(),
+                                 String.format("Number of partitions %d supposed to be more than number of token ranges %d",
+                                               nrPartitions, ring.getRangeMap().asMapOfRanges().keySet().size()));
+        Preconditions.checkState(nrPartitions >= ring.getTokenRanges().keySet().size(),
+                                 String.format("Number of partitions %d supposed to be more than number of instances %d",
+                                               nrPartitions, ring.getTokenRanges().keySet().size()));
+        Preconditions.checkState(partitionMap.asMapOfRanges().keySet().size() == reversePartitionMap.keySet().size(),
+                                 String.format("You must be kidding me! Partition map %d and reverse map %d are not of same size",
+                                               partitionMap.asMapOfRanges().keySet().size(), reversePartitionMap.keySet().size()));
+    }
+
+    private void writeObject(ObjectOutputStream out) throws IOException
+    {
+        out.defaultWriteObject();
+        HashMap<Range<BigInteger>, Integer> partitionEntires = new HashMap<>();
+        partitionMap.asMapOfRanges().forEach(partitionEntires::put);
+        out.writeObject(partitionEntires);
+    }
+
+    @SuppressWarnings("unchecked")
+    private void readObject(ObjectInputStream in) throws ClassNotFoundException, IOException
+    {
+        in.defaultReadObject();
+        HashMap<Range<BigInteger>, Integer> partitionEntires = (HashMap<Range<BigInteger>, Integer>) in.readObject();
+        partitionMap = TreeRangeMap.create();
+        reversePartitionMap = new HashMap<>();
+        partitionEntires.forEach((r, i) -> {
+            partitionMap.put(r, i);
+            reversePartitionMap.put(i, r);
+            nrPartitions++;
+        });
+        LOGGER.info("Partition map " + partitionMap);
+        LOGGER.info("Reverse partition map " + reversePartitionMap);
+        LOGGER.info("Number of partitions {}", nrPartitions);
+    }
+
+    // In order to best utilize the number of Spark cores while minimizing the number of commit calls,
+    // we calculate the number of splits that will just match or exceed the total number of available Spark cores.
+    // NOTE: The actual number of partitions that result from this should always be at least
+    //       the number of token ranges times the number of splits, but can be slightly more.
+    public int calculateSplits(CassandraRing<RingInstance> ring,
+                               Integer numberSplits,
+                               int defaultParallelism,
+                               Integer cores)
+    {
+        if (numberSplits >= 0)
+        {
+            return numberSplits;
+        }
+        int tasksToRun = Math.max(cores, defaultParallelism);
+        Map<Range<BigInteger>, List<RingInstance>> rangeListMap = ring.getRangeMap().asMapOfRanges();
+        LOGGER.info("Initial ranges: {}", rangeListMap);
+        int ranges = rangeListMap.size();
+        LOGGER.info("Number of ranges: {}", ranges);
+        int calculatedSplits = divCeil(tasksToRun, ranges);
+        LOGGER.info("Calculated number of splits as {}", calculatedSplits);
+        return calculatedSplits;
+    }
+
+    int divCeil(int a, int b)
+    {
+        return (a + b - 1) / b;
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/Tokenizer.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/Tokenizer.java
new file mode 100644
index 0000000..cdd20c2
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/Tokenizer.java
@@ -0,0 +1,47 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.spark.bulkwriter.token.TokenUtils;
+import org.apache.cassandra.spark.data.partitioner.Partitioner;
+
+public class Tokenizer implements Serializable
+{
+    private final TokenUtils tokenUtils;
+    private final TableSchema tableSchema;
+
+    public Tokenizer(BulkWriterContext writerContext)
+    {
+        this.tableSchema = writerContext.schema().getTableSchema();
+        this.tokenUtils = new TokenUtils(tableSchema.partitionKeyColumns,
+                                         tableSchema.partitionKeyColumnTypes,
+                                         writerContext.cluster().getPartitioner() == Partitioner.Murmur3Partitioner);
+    }
+
+    public DecoratedKey getDecoratedKey(Object[] columns)
+    {
+        Object[] keyColumns = tableSchema.getKeyColumns(columns);
+        ByteBuffer key = tokenUtils.getCompositeKey(keyColumns);
+        return new DecoratedKey(tokenUtils.getToken(key), key);
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/WriteMode.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/WriteMode.java
new file mode 100644
index 0000000..7d488fa
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/WriteMode.java
@@ -0,0 +1,26 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+public enum WriteMode
+{
+    INSERT,
+    DELETE_PARTITION
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/WriterOption.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/WriterOption.java
new file mode 100644
index 0000000..65bf8e3
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/WriterOption.java
@@ -0,0 +1,25 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+public interface WriterOption
+{
+    String name();
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/WriterOptions.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/WriterOptions.java
new file mode 100644
index 0000000..7b48df8
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/WriterOptions.java
@@ -0,0 +1,47 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+public enum WriterOptions implements WriterOption
+{
+    SIDECAR_INSTANCES,
+    KEYSPACE,
+    TABLE,
+    BULK_WRITER_CL,
+    LOCAL_DC,
+    NUMBER_SPLITS,
+    BATCH_SIZE,
+    COMMIT_THREADS_PER_INSTANCE,
+    COMMIT_BATCH_SIZE,
+    VALIDATE_SSTABLES,
+    SKIP_EXTENDED_VERIFY,
+    WRITE_MODE,
+    KEYSTORE_PASSWORD,
+    KEYSTORE_PATH,
+    KEYSTORE_BASE64_ENCODED,
+    KEYSTORE_TYPE,
+    TRUSTSTORE_PASSWORD,
+    TRUSTSTORE_TYPE,
+    TRUSTSTORE_PATH,
+    TRUSTSTORE_BASE64_ENCODED,
+    SIDECAR_PORT,
+    ROW_BUFFER_MODE,
+    SSTABLE_DATA_SIZE_IN_MB
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/token/CassandraRing.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/token/CassandraRing.java
new file mode 100644
index 0000000..22c0519
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/token/CassandraRing.java
@@ -0,0 +1,291 @@
+/*
+ * 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.cassandra.spark.bulkwriter.token;
+
+import java.io.IOException;
+import java.io.NotSerializableException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.Serializable;
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.TreeRangeMap;
+import org.apache.commons.lang.builder.HashCodeBuilder;
+
+import org.apache.cassandra.spark.bulkwriter.RingInstance;
+import org.apache.cassandra.spark.common.model.CassandraInstance;
+import org.apache.cassandra.spark.data.ReplicationFactor;
+import org.apache.cassandra.spark.data.partitioner.Partitioner;
+
+import static org.apache.cassandra.spark.data.ReplicationFactor.ReplicationStrategy;
+
+/**
+ * CassandraRing is designed to have one unique way of handling Cassandra token/topology information across all Cassandra
+ * tooling. This class is made Serializable so it's easy to use it from Hadoop/Spark. As Cassandra token ranges are
+ * dependent on Replication strategy, ring makes sense for a specific keyspace only. It is made to be immutable for the
+ * sake of simplicity.
+ *
+ * Token ranges are calculated assuming Cassandra racks are not being used, but controlled by assigning tokens properly.
+ * This is the case for now with managed clusters. We should re-think about this if it changes in future.
+ *
+ * {@link #equals(Object)} and {@link #hashCode()} doesn't take {@link #replicas} and {@link #tokenRangeMap} into
+ * consideration as they are just derived fields.
+ *
+ * {@link CassandraInstance} doesn't extend {@link Serializable}. So, it is possible to use {@link CassandraRing} with
+ * non serializable {@link CassandraInstance}. If we try to serialize ring with those instances, you will see
+ * {@link NotSerializableException}. One of the serializable implementation of {@link CassandraInstance} is at
+ * {@link RingInstance}.
+ *
+ * @param <Instance> ring instance type
+ */
+public class CassandraRing<Instance extends CassandraInstance> implements Serializable
+{
+    private final Partitioner partitioner;
+    private final String keyspace;
+    private transient ReplicationFactor replicationFactor;
+    private transient ArrayList<Instance> instances;
+
+    private transient RangeMap<BigInteger, List<Instance>> replicas;
+    private transient Multimap<Instance, Range<BigInteger>> tokenRangeMap;
+
+    /**
+     * Add a replica with given range to replicaMap (RangeMap pointing to replicas).
+     *
+     * replicaMap starts with full range (representing complete ring) with empty list of replicas. So, it is
+     * guaranteed that range will match one or many ranges in replicaMap.
+     *
+     * Scheme to add a new replica for a range:
+     *   * Find overlapping rangeMap entries from replicaMap
+     *   * For each overlapping range, create new replica list by adding new replica to the existing list and add it
+     *     back to replicaMap.
+     */
+    private static <Instance extends CassandraInstance> void addReplica(Instance replica,
+                                                                        Range<BigInteger> range,
+                                                                        RangeMap<BigInteger, List<Instance>> replicaMap)
+    {
+        Preconditions.checkArgument(range.lowerEndpoint().compareTo(range.upperEndpoint()) <= 0,
+                                    "Range calculations assume range is not wrapped");
+
+        RangeMap<BigInteger, List<Instance>> replicaRanges = replicaMap.subRangeMap(range);
+        RangeMap<BigInteger, List<Instance>> mappingsToAdd = TreeRangeMap.create();
+
+        replicaRanges.asMapOfRanges().forEach((key, value) -> {
+            List<Instance> replicas = new ArrayList<>(value);
+            replicas.add(replica);
+            mappingsToAdd.put(key, replicas);
+        });
+        replicaMap.putAll(mappingsToAdd);
+    }
+
+    /**
+     * This is the only way of calculating {@link #replicas} and {@link #tokenRangeMap} either it is from Constructor or
+     * from {@link #readObject(ObjectInputStream)}
+     */
+    private void setupTokenRangeMap()
+    {
+        replicas = TreeRangeMap.create();
+        tokenRangeMap = ArrayListMultimap.create();
+
+        // Calculate instance to token ranges mapping
+        if (replicationFactor.getReplicationStrategy() == ReplicationStrategy.SimpleStrategy)
+        {
+            tokenRangeMap.putAll(RangeUtils.calculateTokenRanges(instances,
+                                                                 replicationFactor.getTotalReplicationFactor(),
+                                                                 partitioner));
+        }
+        else if (replicationFactor.getReplicationStrategy() == ReplicationStrategy.NetworkTopologyStrategy)
+        {
+            for (String dataCenter : getDataCenters())
+            {
+                int rf = replicationFactor.getOptions().get(dataCenter);
+                if (rf == 0)
+                {
+                    // Apparently, it is valid to have zero replication factor in Cassandra
+                    continue;
+                }
+                List<Instance> dcInstances = instances.stream()
+                                                      .filter(instance -> instance.getDataCenter().matches(dataCenter))
+                                                      .collect(Collectors.toList());
+                tokenRangeMap.putAll(RangeUtils.calculateTokenRanges(dcInstances,
+                                                                     replicationFactor.getOptions().get(dataCenter),
+                                                                     partitioner));
+            }
+        }
+        else
+        {
+            throw new UnsupportedOperationException("Unsupported replication strategy");
+        }
+
+        // Calculate token range to replica mapping
+        replicas.put(Range.closed(partitioner.minToken(), partitioner.maxToken()), Collections.emptyList());
+        tokenRangeMap.asMap().forEach((instance, ranges) -> ranges.forEach(range -> addReplica(instance, range, replicas)));
+    }
+
+    public CassandraRing(Partitioner partitioner,
+                         String keyspace,
+                         ReplicationFactor replicationFactor,
+                         Collection<Instance> instances)
+    {
+        this.partitioner = partitioner;
+        this.keyspace = keyspace;
+        this.replicationFactor = replicationFactor;
+        this.instances = instances.stream()
+                                  .sorted(Comparator.comparing(instance -> new BigInteger(instance.getToken())))
+                                  .collect(Collectors.toCollection(ArrayList::new));
+
+        this.setupTokenRangeMap();
+    }
+
+    public Partitioner getPartitioner()
+    {
+        return partitioner;
+    }
+
+    public String getKeyspace()
+    {
+        return keyspace;
+    }
+
+    public ReplicationFactor getReplicationFactor()
+    {
+        return replicationFactor;
+    }
+
+    public Collection<Instance> getInstances()
+    {
+        return instances;
+    }
+
+    public RangeMap<BigInteger, List<Instance>> getRangeMap()
+    {
+        return replicas;
+    }
+
+    public RangeMap<BigInteger, List<Instance>> getSubRanges(Range<BigInteger> tokenRange)
+    {
+        return replicas.subRangeMap(tokenRange);
+    }
+
+    public Collection<BigInteger> getTokens()
+    {
+        return instances.stream()
+                        .map(CassandraInstance::getToken)
+                        .map(BigInteger::new)
+                        .sorted()
+                        .collect(Collectors.toList());
+    }
+
+    public Collection<Range<BigInteger>> getTokenRanges(Instance instance)
+    {
+        return tokenRangeMap.get(instance);
+    }
+
+    public Multimap<Instance, Range<BigInteger>> getTokenRanges()
+    {
+        return tokenRangeMap;
+    }
+
+    public Collection<String> getDataCenters()
+    {
+        return replicationFactor.getReplicationStrategy() == ReplicationStrategy.SimpleStrategy
+                ? Collections.emptySet()
+                : replicationFactor.getOptions().keySet();
+    }
+
+    @Override
+    public boolean equals(Object other)
+    {
+        if (this == other)
+        {
+            return true;
+        }
+        if (other == null || this.getClass() != other.getClass())
+        {
+            return false;
+        }
+
+        CassandraRing<?> that = (CassandraRing<?>) other;
+        return this.partitioner == that.partitioner
+            && this.keyspace.equals(that.keyspace)
+            && this.replicationFactor.getReplicationStrategy() == that.replicationFactor.getReplicationStrategy()
+            && this.replicationFactor.getOptions().equals(that.replicationFactor.getOptions())
+            && this.instances.equals(that.instances);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return new HashCodeBuilder()
+               .append(partitioner)
+               .append(keyspace)
+               .append(replicationFactor)
+               .append(instances)
+               .hashCode();
+    }
+
+    @SuppressWarnings("unchecked")
+    private void readObject(ObjectInputStream in) throws ClassNotFoundException, IOException
+    {
+        in.defaultReadObject();
+
+        // Get replication factor from options - HashMap
+        replicationFactor = new ReplicationFactor((HashMap<String, String>) in.readObject());
+
+        // Read list of SerializableCasInstance
+        instances = (ArrayList<Instance>) in.readObject();
+
+        setupTokenRangeMap();
+    }
+
+    @SuppressWarnings("unchecked")
+    private void writeObject(ObjectOutputStream out) throws IOException
+    {
+        out.defaultWriteObject();
+
+        // ReplicationFactor can be built from options. Build and serialize options
+        HashMap<String, String> replicationOptions = new HashMap<>();
+        replicationOptions.put("class", replicationFactor.getReplicationStrategy() == ReplicationStrategy.SimpleStrategy
+                                         ? "org.apache.cassandra.locator.SimpleStrategy"
+                                         : "org.apache.cassandra.locator.NetworkTopologyStrategy");
+        replicationFactor.getOptions().keySet().forEach(option -> replicationOptions.put(option, replicationFactor.getOptions().get(option).toString()));
+        out.writeObject(replicationOptions);
+
+        if (!instances.isEmpty() && !(instances.get(0) instanceof Serializable))
+        {
+            throw new NotSerializableException(String.format("Cassandra instances(%s) are not Serializable",
+                                                             instances.get(0).getClass()));
+        }
+
+        // Write instances
+        out.writeObject(instances);
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/token/ConsistencyLevel.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/token/ConsistencyLevel.java
new file mode 100644
index 0000000..9153cbd
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/token/ConsistencyLevel.java
@@ -0,0 +1,175 @@
+/*
+ * 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.cassandra.spark.bulkwriter.token;
+
+import java.util.Collection;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.cassandra.spark.common.model.CassandraInstance;
+import org.apache.cassandra.spark.data.ReplicationFactor;
+
+public interface ConsistencyLevel
+{
+    boolean isLocal();
+
+    boolean checkConsistency(Collection<? extends CassandraInstance> failedInsts, ReplicationFactor replicationFactor, String localDC);
+
+    enum CL implements ConsistencyLevel
+    {
+        ALL
+        {
+            @Override
+            public boolean isLocal()
+            {
+                return false;
+            }
+
+            @Override
+            public boolean checkConsistency(Collection<? extends CassandraInstance> failedInsts,
+                                            ReplicationFactor replicationFactor,
+                                            String localDC)
+            {
+                return failedInsts.isEmpty();
+            }
+        },
+        EACH_QUORUM
+        {
+            @Override
+            public boolean isLocal()
+            {
+                return false;
+            }
+
+            @Override
+            public boolean checkConsistency(Collection<? extends CassandraInstance> failedInsts,
+                                            ReplicationFactor replicationFactor,
+                                            String localDC)
+            {
+                Preconditions.checkArgument(replicationFactor.getReplicationStrategy() != ReplicationFactor.ReplicationStrategy.SimpleStrategy,
+                                            "EACH_QUORUM doesn't make sense for SimpleStrategy keyspaces");
+
+                for (String datacenter : replicationFactor.getOptions().keySet())
+                {
+                    int rf = replicationFactor.getOptions().get(datacenter);
+                    if (failedInsts.stream()
+                                   .filter(instance -> instance.getDataCenter().matches(datacenter))
+                                   .count() > (rf - (rf / 2 + 1)))
+                    {
+                        return false;
+                    }
+                }
+
+                return true;
+            }
+        },
+        QUORUM
+        {
+            @Override
+            public boolean isLocal()
+            {
+                return false;
+            }
+
+            @Override
+            public boolean checkConsistency(Collection<? extends CassandraInstance> failedInsts,
+                                            ReplicationFactor replicationFactor,
+                                            String localDC)
+            {
+                int rf = replicationFactor.getTotalReplicationFactor();
+                return failedInsts.size() <= (rf - (rf / 2 + 1));
+            }
+        },
+        LOCAL_QUORUM
+        {
+            @Override
+            public boolean isLocal()
+            {
+                return true;
+            }
+
+            @Override
+            public boolean checkConsistency(Collection<? extends CassandraInstance> failedInsts,
+                                            ReplicationFactor replicationFactor,
+                                            String localDC)
+            {
+                Preconditions.checkArgument(replicationFactor.getReplicationStrategy() != ReplicationFactor.ReplicationStrategy.SimpleStrategy,
+                                            "LOCAL_QUORUM doesn't make sense for SimpleStrategy keyspaces");
+
+                int rf = replicationFactor.getOptions().get(localDC);
+                return failedInsts.stream().filter(instance -> instance.getDataCenter().matches(localDC)).count() <= (rf - (rf / 2 + 1));
+            }
+        },
+        ONE
+        {
+            @Override
+            public boolean isLocal()
+            {
+                return false;
+            }
+
+            @Override
+            public boolean checkConsistency(Collection<? extends CassandraInstance> failedInsts,
+                                            ReplicationFactor replicationFactor,
+                                            String localDC)
+            {
+                int rf = replicationFactor.getTotalReplicationFactor();
+                return failedInsts.size() <= rf - 1;
+            }
+        },
+        TWO
+        {
+            @Override
+            public boolean isLocal()
+            {
+                return false;
+            }
+
+            @Override
+            public boolean checkConsistency(Collection<? extends CassandraInstance> failedInsts,
+                                            ReplicationFactor replicationFactor,
+                                            String localDC)
+            {
+                int rf = replicationFactor.getTotalReplicationFactor();
+                return failedInsts.size() <= rf - 2;
+            }
+        },
+        LOCAL_ONE
+        {
+            @Override
+            public boolean isLocal()
+            {
+                return true;
+            }
+
+            @Override
+            public boolean checkConsistency(Collection<? extends CassandraInstance> failedInsts,
+                                            ReplicationFactor replicationFactor,
+                                            String localDC)
+            {
+                Preconditions.checkArgument(replicationFactor.getReplicationStrategy() != ReplicationFactor.ReplicationStrategy.SimpleStrategy,
+                                            "LOCAL_QUORUM doesn't make sense for SimpleStrategy keyspaces");
+
+                int rf = replicationFactor.getOptions().get(localDC);
+                return failedInsts.stream().filter(instance -> instance.getDataCenter().matches(localDC)).count() <= (rf - 1);
+            }
+        }
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/token/RangeUtils.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/token/RangeUtils.java
new file mode 100644
index 0000000..3165aa2
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/token/RangeUtils.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.cassandra.spark.bulkwriter.token;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.BoundType;
+import com.google.common.collect.Multimap;
+import com.google.common.collect.Range;
+
+import org.apache.cassandra.spark.common.model.CassandraInstance;
+import org.apache.cassandra.spark.data.partitioner.Partitioner;
+
+/**
+ * Common Cassandra range operations on Guava ranges. Assumes ranges are not wrapped around.
+ * It's the responsibility of caller to unwrap ranges. For example, [100, 1] should become
+ * [100, MAX] and [MIN, 1]. MIN and MAX depend on {@link Partitioner}.
+ */
+public final class RangeUtils
+{
+    private RangeUtils()
+    {
+    }
+
+    public static BigInteger sizeOf(Range<BigInteger> range)
+    {
+        Preconditions.checkArgument(range.lowerEndpoint().compareTo(range.upperEndpoint()) <= 0,
+                                    "RangeUtils assume ranges are not wrap-around");
+
+        if (range.isEmpty())
+        {
+            return BigInteger.ZERO;
+        }
+
+        BigInteger size = range.upperEndpoint().subtract(range.lowerEndpoint()).add(BigInteger.ONE);
+
+        if (range.lowerBoundType() == BoundType.OPEN)
+        {
+            size = size.subtract(BigInteger.ONE);
+        }
+
+        if (range.upperBoundType() == BoundType.OPEN)
+        {
+            size = size.subtract(BigInteger.ONE);
+        }
+
+        return size;
+    }
+
+    /**
+     * Splits the given range into equal sized small ranges. Number of splits can be controlled by
+     * nrSplits. If nrSplits are smaller than size of the range, split size would be set to 1.
+     *
+     * This is best effort scheme, nrSplits not necessarily as promised and not all splits may not be
+     * exact same size.
+     *
+     * @param range the range to split
+     * @param nrSplits the number of sub-ranges into which the range should be divided
+     * @return a list of sub-ranges
+     */
+    public static List<Range<BigInteger>> split(Range<BigInteger> range, int nrSplits)
+    {
+        Preconditions.checkArgument(range.lowerEndpoint().compareTo(range.upperEndpoint()) <= 0,
+                                    "RangeUtils assume ranges are not wrap-around");
+
+        if (range.isEmpty())
+        {
+            return Collections.emptyList();
+        }
+
+        Preconditions.checkArgument(nrSplits >= 1, "nrSplits must be greater than or equal to 1");
+
+        // Make sure split size is not 0
+        BigInteger splitSize = sizeOf(range).divide(BigInteger.valueOf(nrSplits));
+        if (splitSize.compareTo(BigInteger.ZERO) == 0)
+        {
+            splitSize = BigInteger.ONE;
+        }
+
+        // Start from range lower endpoint and spit ranges of size splitSize, until we cross the range
+        BigInteger nextLowerEndpoint = range.lowerBoundType() == BoundType.CLOSED
+                ? range.lowerEndpoint()
+                : range.lowerEndpoint().add(BigInteger.ONE);
+        List<Range<BigInteger>> splits = new ArrayList<>();
+        while (range.contains(nextLowerEndpoint))
+        {
+            BigInteger upperEndpoint = nextLowerEndpoint.add(splitSize);
+            splits.add(range.intersection(Range.closedOpen(nextLowerEndpoint, upperEndpoint)));
+            nextLowerEndpoint = upperEndpoint;
+        }
+
+        return splits;
+    }
+
+    public static <Instance extends CassandraInstance> Multimap<Instance, Range<BigInteger>> calculateTokenRanges(
+            List<Instance> instances,
+            int replicationFactor,
+            Partitioner partitioner)
+    {
+        Preconditions.checkArgument(replicationFactor != 0, "Calculation token ranges wouldn't work with RF 0");
+        Preconditions.checkArgument(instances.size() == 0 || replicationFactor <= instances.size(),
+                                    "Calculation token ranges wouldn't work when RF (" + replicationFactor
+                                  + ") is greater than number of Cassandra instances " + instances.size());
+        Multimap<Instance, Range<BigInteger>> tokenRanges = ArrayListMultimap.create();
+        for (int index = 0; index < instances.size(); index++)
+        {
+            Instance instance = instances.get(index);
+            int disjointReplica = ((instances.size() + index) - replicationFactor) % instances.size();
+            BigInteger rangeStart = new BigInteger(instances.get(disjointReplica).getToken());
+            BigInteger rangeEnd = new BigInteger(instance.getToken());
+
+            // If start token is not strictly smaller than end token we are looking at a wrap around range, split it
+            if (rangeStart.compareTo(rangeEnd) >= 0)
+            {
+                tokenRanges.put(instance, Range.range(rangeStart, BoundType.OPEN, partitioner.maxToken(), BoundType.CLOSED));
+                tokenRanges.put(instance, Range.range(partitioner.minToken(), BoundType.CLOSED, rangeEnd, BoundType.CLOSED));
+            }
+            else
+            {
+                tokenRanges.put(instance, Range.openClosed(rangeStart, rangeEnd));
+            }
+        }
+
+        return tokenRanges;
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/token/ReplicaAwareFailureHandler.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/token/ReplicaAwareFailureHandler.java
new file mode 100644
index 0000000..e57c571
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/token/ReplicaAwareFailureHandler.java
@@ -0,0 +1,134 @@
+/*
+ * 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.cassandra.spark.bulkwriter.token;
+
+import java.math.BigInteger;
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.TreeRangeMap;
+
+import org.apache.cassandra.spark.common.model.CassandraInstance;
+
+public class ReplicaAwareFailureHandler<Instance extends CassandraInstance>
+{
+    private final CassandraRing<Instance> ring;
+    private final RangeMap<BigInteger, Multimap<Instance, String>> failedRangesMap = TreeRangeMap.create();
+
+    public ReplicaAwareFailureHandler(CassandraRing<Instance> ring)
+    {
+        this.ring = ring;
+        failedRangesMap.put(Range.closed(ring.getPartitioner().minToken(),
+                                         ring.getPartitioner().maxToken()),
+                            ArrayListMultimap.create());
+    }
+
+    /**
+     * Adds a new token range as a failed token range, with errors on given instance.
+     *
+     * It's guaranteed that failedRangesMap has overlapping ranges for the range we are trying to insert (Check
+     * constructor, we are adding complete ring first).
+     *
+     * So the scheme is to get list of overlapping ranges first. For each overlapping range get the failure map.
+     * Make a copy of the map and add new failure to this map. It's important we make the copy and not use the
+     * one returned from failedRangesMap map. As our range could be overlapping partially and the map could be used
+     * by other range.
+     *
+     * @param tokenRange the range which failed
+     * @param casInstance the instance on which the range failed
+     * @param errMessage the error that occurred for this particular range/instance pair
+     */
+    public void addFailure(Range<BigInteger> tokenRange, Instance casInstance, String errMessage)
+    {
+        RangeMap<BigInteger, Multimap<Instance, String>> overlappingFailures = failedRangesMap.subRangeMap(tokenRange);
+        RangeMap<BigInteger, Multimap<Instance, String>> mappingsToAdd = TreeRangeMap.create();
+
+        for (Map.Entry<Range<BigInteger>, Multimap<Instance, String>> entry : overlappingFailures.asMapOfRanges().entrySet())
+        {
+            Multimap<Instance, String> newErrorMap = ArrayListMultimap.create(entry.getValue());
+
+            newErrorMap.put(casInstance, errMessage);
+            mappingsToAdd.put(entry.getKey(), newErrorMap);
+        }
+        failedRangesMap.putAll(mappingsToAdd);
+    }
+
+    public boolean hasFailed(ConsistencyLevel consistencyLevel, String localDC)
+    {
+        return !getFailedEntries(consistencyLevel, localDC).isEmpty();
+    }
+
+    @SuppressWarnings("unused")  // Convenience method can become useful in the future
+    public Collection<Range<BigInteger>> getFailedRanges(ConsistencyLevel consistencyLevel, String localDC)
+    {
+        return getFailedEntries(consistencyLevel, localDC).stream()
+                .map(AbstractMap.SimpleEntry::getKey)
+                .collect(Collectors.toList());
+    }
+
+    @SuppressWarnings("unused")  // Convenience method can become useful in the future
+    public Multimap<Instance, String> getFailedInstances(ConsistencyLevel consistencyLevel, String localDC)
+    {
+        Multimap<Instance, String> failedInstances = ArrayListMultimap.create();
+        getFailedEntries(consistencyLevel, localDC).stream()
+                .flatMap(failedMultiEntry -> failedMultiEntry.getValue().entries().stream())
+                .forEach(failedEntry -> failedInstances.put(failedEntry.getKey(), failedEntry.getValue()));
+
+        return failedInstances;
+    }
+
+    public Collection<AbstractMap.SimpleEntry<Range<BigInteger>, Multimap<Instance, String>>> getFailedEntries(
+            ConsistencyLevel consistencyLevel,
+            String localDC)
+    {
+        List<AbstractMap.SimpleEntry<Range<BigInteger>, Multimap<Instance, String>>> failedEntries = new ArrayList<>();
+
+        for (Map.Entry<Range<BigInteger>, Multimap<Instance, String>> failedRangeEntry : failedRangesMap.asMapOfRanges().entrySet())
+        {
+            Multimap<Instance, String> errorMap = failedRangeEntry.getValue();
+            Collection<Instance> failedInstances = errorMap.keySet().stream()
+                    .filter(instance -> !errorMap.get(instance).isEmpty())
+                    .collect(Collectors.toList());
+            if (!consistencyLevel.checkConsistency(failedInstances, ring.getReplicationFactor(), localDC))
+            {
+                failedEntries.add(new AbstractMap.SimpleEntry<>(failedRangeEntry.getKey(), failedRangeEntry.getValue()));
+            }
+        }
+
+        return failedEntries;
+    }
+
+    public String toString()
+    {
+        return "CassandraRing: Tokens: " + ring.getTokens().stream()
+                                                           .map(BigInteger::toString)
+                                                           .collect(Collectors.joining(",", "[", "]")) + ", "
+                 + "ReplicationFactor: " + ring.getReplicationFactor().getReplicationStrategy().name() + " "
+                                         + ring.getReplicationFactor().getOptions();
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/token/TokenUtils.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/token/TokenUtils.java
new file mode 100644
index 0000000..b0169ae
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/token/TokenUtils.java
@@ -0,0 +1,164 @@
+/*
+ * 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.cassandra.spark.bulkwriter.token;
+
+import java.io.Serializable;
+import java.math.BigInteger;
+import java.nio.Buffer;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+import org.apache.cassandra.spark.common.schema.ColumnType;
+import org.apache.cassandra.spark.common.schema.ColumnTypes;
+import org.apache.cassandra.spark.data.partitioner.Partitioner;
+
+/**
+ * This is utility class for computing Cassandra token for a CQL row.
+ * This is mainly written to partition Cassandra ring and split it between Spark executors.
+ * And to make sure each executor is only talking to one Cassandra replica set.
+ * This reduces number of SSTables that get created in Cassandra by the bulk writing job.
+ * Fewer SSTables will result in lower read latencies and lower compaction overhead.
+ */
+@SuppressWarnings("WeakerAccess")
+public class TokenUtils implements Serializable
+{
+    private final String[] partitionKeyColumns;
+    private final boolean isMurmur3Partitioner;
+    private final ColumnType[] partitionKeyColumnTypes;
+
+    public TokenUtils(List<String> partitionKeyColumns,
+                      List<ColumnType<?>> partitionKeyColumnTypes,
+                      boolean isMurmur3Partitioner)
+    {
+        this.partitionKeyColumns = partitionKeyColumns.toArray(new String[0]);
+        this.partitionKeyColumnTypes = partitionKeyColumnTypes.toArray(new ColumnType[partitionKeyColumns.size()]);
+        this.isMurmur3Partitioner = isMurmur3Partitioner;
+    }
+
+    // noinspection unchecked
+    private ByteBuffer getByteBuffer(Object columnValue, int partitionKeyColumnIdx)
+    {
+        ColumnType columnType = partitionKeyColumnTypes[partitionKeyColumnIdx];
+        if (columnType == ColumnTypes.UUID)
+        {
+            return columnType.serialize(UUID.fromString((String) columnValue));
+        }
+        else
+        {
+            return columnType.serialize(columnValue);
+        }
+    }
+
+    private BigInteger getToken(ByteBuffer[] byteBuffers)
+    {
+        return getToken(getCompositeKey(byteBuffers));
+    }
+
+    public BigInteger getToken(ByteBuffer key)
+    {
+        return isMurmur3Partitioner ? Partitioner.Murmur3Partitioner.hash(key) : Partitioner.RandomPartitioner.hash(key);
+    }
+
+    public ByteBuffer getCompositeKey(Object[] columnValues)
+    {
+        ByteBuffer[] byteBuffers = new ByteBuffer[partitionKeyColumns.length];
+
+        for (int column = 0; column < partitionKeyColumns.length; column++)
+        {
+            byteBuffers[column] = getByteBuffer(columnValues[column], column);
+        }
+
+        return getCompositeKey(byteBuffers);
+    }
+
+    public ByteBuffer getCompositeKey(ByteBuffer[] byteBuffers)
+    {
+        ByteBuffer key;
+        if (byteBuffers.length == 1)
+        {
+            key = byteBuffers[0];
+        }
+        else
+        {
+            // Calculate length of the key
+            int length = 0;
+            for (ByteBuffer buffer : byteBuffers)
+            {
+                length += 2;
+                length += buffer.remaining();
+                length += 1;
+            }
+
+            // Add buffers one after another
+            key = ByteBuffer.allocate(length);
+            for (ByteBuffer buffer : byteBuffers)
+            {
+                key.putShort((short) buffer.remaining());
+                key.put(buffer);
+                key.put((byte) 0x00);
+            }
+            ((Buffer) key).flip();
+        }
+
+        return key;
+    }
+
+    /**
+     * Calculate Cassandra token for CQL row
+     *
+     * @param columnValues Map of partitioner key columns to column value as Java type
+     * @return Cassandra token
+     */
+    public BigInteger getToken(Map<String, Object> columnValues)
+    {
+        ByteBuffer[] byteBuffers = new ByteBuffer[partitionKeyColumns.length];
+
+        for (int column = 0; column < partitionKeyColumns.length; column++)
+        {
+            byteBuffers[column] = getByteBuffer(columnValues.get(partitionKeyColumns[column]), column);
+        }
+
+        return getToken(byteBuffers);
+    }
+
+    /**
+     * Calculate Cassandra token for CQL row
+     *
+     * @param columnValues list of partitioner key columns in the same order as partition key
+     * @return Cassandra token
+     */
+    public BigInteger getToken(List<Object> columnValues)
+    {
+        return getToken(columnValues.toArray());
+    }
+
+    /**
+     * Calculate Cassandra token for CQL row
+     *
+     * @param columnValues array of partitioner key columns in the same order as partition key
+     * @return Cassandra token
+     */
+    public BigInteger getToken(Object[] columnValues)
+    {
+        return getToken(getCompositeKey(columnValues));
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/util/FastByteOperations.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/util/FastByteOperations.java
new file mode 100644
index 0000000..12a158a
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/util/FastByteOperations.java
@@ -0,0 +1,267 @@
+/*
+ * 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.cassandra.spark.bulkwriter.util;
+
+import java.lang.reflect.Field;
+import java.nio.Buffer;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.AccessController;
+import java.security.PrivilegedAction;
+
+import com.google.common.primitives.Longs;
+import com.google.common.primitives.UnsignedBytes;
+import com.google.common.primitives.UnsignedLongs;
+
+import sun.misc.Unsafe;  // CHECKSTYLE IGNORE: Used with care to speed byte operations up
+
+/**
+ * Utility code to do optimized byte-array comparison.
+ * This is borrowed and slightly modified from Guava's {@link UnsignedBytes}
+ * class to be able to compare arrays that start at non-zero offsets.
+ * NOTE: This was lifted from Cassandra 2.1 and reduced to the minimum number of
+ * methods necessary for the Spark Bulk Writer. Tests in Cassandra, not ported here
+ * as more copy/paste didn't seem to be valuable. Also tested via DecoratedKeyTest.
+ */
+public final class FastByteOperations
+{
+    private FastByteOperations()
+    {
+        throw new IllegalStateException(getClass() + " is static utility class and shall not be instantiated");
+    }
+
+    /**
+     * Lexicographically compare two byte arrays
+     *
+     * @param buffer1 the first bytebuffer
+     * @param buffer2 the second byteburrer
+     * @return standard java comparison return value (-1 for &lt;, 0 for ==, and 1 for &gt;)
+     */
+    public static int compareUnsigned(ByteBuffer buffer1, ByteBuffer buffer2)
+    {
+        return BestHolder.BEST.compare(buffer1, buffer2);
+    }
+
+    public interface ByteOperations
+    {
+        int compare(ByteBuffer buffer1, ByteBuffer buffer2);
+    }
+
+    /**
+     * Provides a lexicographical comparer implementation; either a Java
+     * implementation or a faster implementation based on {@link Unsafe}.
+     *
+     * Uses reflection to gracefully fall back to the Java implementation if
+     * {@code Unsafe} isn't available.
+     */
+    private static class BestHolder
+    {
+        static final String UNSAFE_COMPARER_NAME = FastByteOperations.class.getName() + "$UnsafeOperations";
+        static final ByteOperations BEST = getBest();
+
+        /**
+         * Returns the Unsafe-using Comparer, or falls back to the pure-Java implementation if unable to do so
+         */
+        static ByteOperations getBest()
+        {
+            String arch = System.getProperty("os.arch");
+            boolean unaligned = arch.equals("i386") || arch.equals("x86") || arch.equals("x86_64") || arch.equals("amd64");
+            if (!unaligned)
+            {
+                return new PureJavaOperations();
+            }
+            try
+            {
+                Class<?> theClass = Class.forName(UNSAFE_COMPARER_NAME);
+
+                // Yes, UnsafeComparer does implement Comparer<byte[]>
+                @SuppressWarnings("unchecked")
+                ByteOperations comparer = (ByteOperations) theClass.getConstructor().newInstance();
+                return comparer;
+            }
+            catch (Throwable throwable)
+            {
+                // Ensure we really catch *everything*
+                return new PureJavaOperations();
+            }
+        }
+
+    }
+
+    public static final class UnsafeOperations implements ByteOperations
+    {
+        static final Unsafe UNSAFE;
+        /**
+         * The offset to the first element in a byte array
+         */
+        static final long BYTE_ARRAY_BASE_OFFSET;
+        static final long DIRECT_BUFFER_ADDRESS_OFFSET;
+
+        static
+        {
+            UNSAFE = (Unsafe) AccessController.doPrivileged((PrivilegedAction<Object>) () -> {
+                try
+                {
+                    Field field = Unsafe.class.getDeclaredField("UNSAFE");
+                    field.setAccessible(true);
+                    return field.get(null);
+                }
+                catch (NoSuchFieldException | IllegalAccessException exception)
+                {
+                    // It doesn't matter what we throw; it's swallowed in getBest()
+                    throw new Error();
+                }
+            });
+
+            try
+            {
+                BYTE_ARRAY_BASE_OFFSET = UNSAFE.arrayBaseOffset(byte[].class);
+                DIRECT_BUFFER_ADDRESS_OFFSET = UNSAFE.objectFieldOffset(Buffer.class.getDeclaredField("address"));
+            }
+            catch (Exception exception)
+            {
+                throw new AssertionError(exception);
+            }
+
+            // Sanity check - this should never fail
+            if (UNSAFE.arrayIndexScale(byte[].class) != 1)
+            {
+                throw new AssertionError();
+            }
+        }
+
+        static final boolean BIG_ENDIAN = ByteOrder.nativeOrder().equals(ByteOrder.BIG_ENDIAN);
+
+        public int compare(ByteBuffer buffer1, ByteBuffer buffer2)
+        {
+            return compareTo(buffer1, buffer2);
+        }
+
+        static int compareTo(ByteBuffer buffer1, ByteBuffer buffer2)
+        {
+            Object object1;
+            long offset1;
+            int length1;
+            if (buffer1.hasArray())
+            {
+                object1 = buffer1.array();
+                offset1 = BYTE_ARRAY_BASE_OFFSET + buffer1.arrayOffset();
+            }
+            else
+            {
+                object1 = null;
+                offset1 = UNSAFE.getLong(buffer1, DIRECT_BUFFER_ADDRESS_OFFSET);
+            }
+            offset1 += buffer1.position();
+            length1 = buffer1.remaining();
+            return compareTo(object1, offset1, length1, buffer2);
+        }
+
+        static int compareTo(Object buffer1, long offset1, int length1, ByteBuffer buffer2)
+        {
+            Object object2;
+            long offset2;
+
+            int position = buffer2.position();
+            int limit = buffer2.limit();
+            if (buffer2.hasArray())
+            {
+                object2 = buffer2.array();
+                offset2 = BYTE_ARRAY_BASE_OFFSET + buffer2.arrayOffset();
+            }
+            else
+            {
+                object2 = null;
+                offset2 = UNSAFE.getLong(buffer2, DIRECT_BUFFER_ADDRESS_OFFSET);
+            }
+            int length2 = limit - position;
+            offset2 += position;
+
+            return compareTo(buffer1, offset1, length1, object2, offset2, length2);
+        }
+
+        /**
+         * Lexicographically compare two arrays
+         *
+         * @param buffer1 left operand: a byte[] or null
+         * @param buffer2 right operand: a byte[] or null
+         * @param offset1 Where to start comparing in the left buffer
+         *                (pure memory address if buffer1 is null, or relative otherwise)
+         * @param offset2 Where to start comparing in the right buffer
+         *                (pure memory address if buffer1 is null, or relative otherwise)
+         * @param length1 How much to compare from the left buffer
+         * @param length2 How much to compare from the right buffer
+         * @return 0 if equal, < 0 if left is less than right, etc.
+         */
+        static int compareTo(Object buffer1, long offset1, int length1, Object buffer2, long offset2, int length2)
+        {
+            int minLength = Math.min(length1, length2);
+
+            /*
+             * Compare 8 bytes at a time. Benchmarking shows comparing 8 bytes at a
+             * time is no slower than comparing 4 bytes at a time even on 32-bit.
+             * On the other hand, it is substantially faster on 64-bit.
+             */
+            int wordComparisons = minLength & ~7;
+            for (int index = 0; index < wordComparisons; index += Longs.BYTES)
+            {
+                long long1 = UNSAFE.getLong(buffer1, offset1 + index);
+                long long2 = UNSAFE.getLong(buffer2, offset2 + index);
+
+                if (long1 != long2)
+                {
+                    return BIG_ENDIAN ? UnsignedLongs.compare(long1, long2)
+                                      : UnsignedLongs.compare(Long.reverseBytes(long1), Long.reverseBytes(long2));
+                }
+            }
+
+            for (int index = wordComparisons; index < minLength; index++)
+            {
+                int byte1 = UNSAFE.getByte(buffer1, offset1 + index) & 0xFF;
+                int byte2 = UNSAFE.getByte(buffer2, offset2 + index) & 0xFF;
+                if (byte1 != byte2)
+                {
+                    return byte1 - byte2;
+                }
+            }
+
+            return length1 - length2;
+        }
+    }
+
+    public static final class PureJavaOperations implements ByteOperations
+    {
+        public int compare(ByteBuffer buffer1, ByteBuffer buffer2)
+        {
+            int end1 = buffer1.limit();
+            int end2 = buffer2.limit();
+            for (int index1 = buffer1.position(), index2 = buffer2.position(); index1 < end1 && index2 < end2; index1++, index2++)
+            {
+                int byte1 = buffer1.get(index1) & 0xFF;
+                int byte2 = buffer2.get(index2) & 0xFF;
+                if (byte1 != byte2)
+                {
+                    return byte1 - byte2;
+                }
+            }
+            return buffer1.remaining() - buffer2.remaining();
+        }
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/util/SbwJavaSerializer.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/util/SbwJavaSerializer.java
new file mode 100644
index 0000000..fa03ed2
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/util/SbwJavaSerializer.java
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.cassandra.spark.bulkwriter.util;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.ObjectStreamClass;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.KryoException;
+import com.esotericsoftware.kryo.Serializer;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+import com.esotericsoftware.kryo.util.ObjectMap;
+
+/**
+ * Lifted from Kryo 4.0 to fix issues with ObjectInputStream not using the correct class loader
+ * See https://github.com/EsotericSoftware/kryo/blob/19a6b5edee7125fbaf54c64084a8d0e13509920b/src/com/esotericsoftware/kryo/serializers/JavaSerializer.java
+ */
+// noinspection unchecked
+public class SbwJavaSerializer extends Serializer
+{
+    public void write(Kryo kryo, Output output, Object object)
+    {
+        try
+        {
+            ObjectMap graphContext = kryo.getGraphContext();
+            ObjectOutputStream objectStream = (ObjectOutputStream) graphContext.get(this);
+            if (objectStream == null)
+            {
+                objectStream = new ObjectOutputStream(output);
+                graphContext.put(this, objectStream);
+            }
+            objectStream.writeObject(object);
+            objectStream.flush();
+        }
+        catch (Exception exception)
+        {
+            throw new KryoException("Error during Java serialization.", exception);
+        }
+    }
+
+    public Object read(Kryo kryo, Input input, Class type)
+    {
+        try
+        {
+            ObjectMap graphContext = kryo.getGraphContext();
+            ObjectInputStream objectStream = (ObjectInputStream) graphContext.get(this);
+            if (objectStream == null)
+            {
+                objectStream = new ObjectInputStreamWithKryoClassLoader(input, kryo);
+                graphContext.put(this, objectStream);
+            }
+            return objectStream.readObject();
+        }
+        catch (Exception exception)
+        {
+            throw new KryoException("Error during Java deserialization.", exception);
+        }
+    }
+
+    /**
+     * ${@link ObjectInputStream} uses the last user-defined ${@link ClassLoader} which may not be the correct one.
+     * This is a known Java issue and is often solved by using a specific class loader.
+     * See:
+     * https://github.com/apache/spark/blob/v1.6.3/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala#L154
+     * https://issues.apache.org/jira/browse/GROOVY-1627
+     */
+    private static class ObjectInputStreamWithKryoClassLoader extends ObjectInputStream
+    {
+        private final ClassLoader loader;
+
+        ObjectInputStreamWithKryoClassLoader(InputStream in, Kryo kryo) throws IOException
+        {
+            super(in);
+            this.loader = kryo.getClassLoader();
+        }
+
+        @Override
+        protected Class<?> resolveClass(ObjectStreamClass desc)
+        {
+            try
+            {
+                return Class.forName(desc.getName(), false, loader);
+            }
+            catch (ClassNotFoundException exception)
+            {
+                throw new RuntimeException("Class not found: " + desc.getName(), exception);
+            }
+        }
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/util/SbwKryoRegistrator.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/util/SbwKryoRegistrator.java
new file mode 100644
index 0000000..baf66ac
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/util/SbwKryoRegistrator.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.cassandra.spark.bulkwriter.util;
+
+import java.io.Serializable;
+import java.util.Comparator;
+import java.util.Set;
+
+import com.google.common.collect.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.esotericsoftware.kryo.Kryo;
+import org.apache.cassandra.spark.bulkwriter.CassandraBulkWriterContext;
+import org.apache.cassandra.spark.bulkwriter.RingInstance;
+import org.apache.cassandra.spark.bulkwriter.TokenPartitioner;
+import org.apache.cassandra.spark.bulkwriter.token.CassandraRing;
+import org.apache.spark.SparkConf;
+import org.apache.spark.serializer.KryoRegistrator;
+import org.jetbrains.annotations.NotNull;
+
+public class SbwKryoRegistrator implements KryoRegistrator
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(SbwKryoRegistrator.class);
+    protected static final String KRYO_KEY = "spark.kryo.registrator";
+
+    // CHECKSTYLE IGNORE: Despite being static and final, this is a mutable field not to be confused with a constant
+    private static final Set<Class<? extends Serializable>> javaSerializableClasses =
+            Sets.newHashSet(CassandraBulkWriterContext.class,
+                            CassandraRing.class,
+                            TokenPartitioner.class,
+                            RingInstance.class);
+
+    @Override
+    public void registerClasses(@NotNull Kryo kryo)
+    {
+        LOGGER.debug("Registering Spark Bulk Writer classes with Kryo which require use of Java Serializer");
+        // NOTE: The order of calls to `register` matters, so we sort by class name just to make sure we always
+        //       register classess in the same order - HashSet doesn't guarantee its iteration order
+        javaSerializableClasses.stream()
+                .sorted(Comparator.comparing(Class::getCanonicalName))
+                .forEach(javaSerializableClass -> kryo.register(javaSerializableClass, new SbwJavaSerializer()));
+    }
+
+    public static void addJavaSerializableClass(@NotNull Class<? extends Serializable> javaSerializableClass)
+    {
+        javaSerializableClasses.add(javaSerializableClass);
+    }
+
+    public static void setupKryoRegistrator(@NotNull SparkConf configuration)
+    {
+        String registrators = configuration.get(KRYO_KEY, "");
+        String registrator = SbwKryoRegistrator.class.getName();
+        if (!registrators.contains(registrator))
+        {
+            configuration.set(KRYO_KEY, registrators + "," + registrator);
+        }
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/util/ThreadUtil.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/util/ThreadUtil.java
new file mode 100644
index 0000000..7aababb
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/bulkwriter/util/ThreadUtil.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.cassandra.spark.bulkwriter.util;
+
+import java.util.concurrent.ThreadFactory;
+
+import org.jetbrains.annotations.NotNull;
+
+public final class ThreadUtil
+{
+    private ThreadUtil()
+    {
+        throw new IllegalStateException(getClass() + " is static utility class and shall not be instantiated");
+    }
+
+    @NotNull
+    public static ThreadFactory threadFactory(@NotNull String threadName)
+    {
+        return threadFactory(threadName, true);
+    }
+
+    @NotNull
+    public static ThreadFactory threadFactory(@NotNull String threadName, boolean isDaemon)
+    {
+        return runnable -> newThread(runnable, threadName, isDaemon);
+    }
+
+    @NotNull
+    private static Thread newThread(@NotNull Runnable runnable, @NotNull String threadName, boolean isDaemon)
+    {
+        Thread thread = new Thread(runnable, threadName);
+        thread.setDaemon(isDaemon);
+        return thread;
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/cdc/CdcCellIterator.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/cdc/CdcCellIterator.java
new file mode 100644
index 0000000..5afec69
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/cdc/CdcCellIterator.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.cassandra.spark.cdc;
+
+import java.util.List;
+
+import org.apache.cassandra.spark.data.DataLayer;
+import org.apache.cassandra.spark.reader.StreamScanner;
+import org.apache.cassandra.spark.sparksql.SparkCellIterator;
+import org.apache.cassandra.spark.sparksql.filters.CdcOffsetFilter;
+import org.apache.cassandra.spark.sparksql.filters.PartitionKeyFilter;
+import org.apache.spark.sql.types.StructType;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+public class CdcCellIterator extends SparkCellIterator
+{
+    public CdcCellIterator(int partitionId,
+                           @NotNull DataLayer dataLayer,
+                           @Nullable StructType requiredSchema,
+                           @NotNull List<PartitionKeyFilter> partitionKeyFilters,
+                           @Nullable CdcOffsetFilter cdcOffsetFilter)
+    {
+        super(partitionId, dataLayer, requiredSchema, partitionKeyFilters, cdcOffsetFilter);
+    }
+
+    @Override
+    protected StreamScanner openScanner(int partitionId,
+                                        @NotNull List<PartitionKeyFilter> partitionKeyFilters,
+                                        @Nullable CdcOffsetFilter cdcOffsetFilter)
+    {
+        return dataLayer.openCdcScanner(partitionId, cdcOffsetFilter);
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/cdc/CdcRowIterator.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/cdc/CdcRowIterator.java
new file mode 100644
index 0000000..8f44e57
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/cdc/CdcRowIterator.java
@@ -0,0 +1,53 @@
+/*
+ * 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.cassandra.spark.cdc;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.cassandra.spark.data.DataLayer;
+import org.apache.cassandra.spark.sparksql.SparkCellIterator;
+import org.apache.cassandra.spark.sparksql.SparkRowIterator;
+import org.apache.cassandra.spark.sparksql.filters.CdcOffsetFilter;
+import org.apache.cassandra.spark.sparksql.filters.PartitionKeyFilter;
+import org.apache.spark.sql.types.StructType;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+public class CdcRowIterator extends SparkRowIterator
+{
+    public CdcRowIterator(int partitionId,
+                          @NotNull DataLayer dataLayer,
+                          @Nullable StructType requiredSchema,
+                          @Nullable CdcOffsetFilter cdcOffsetFilter)
+    {
+        super(partitionId, dataLayer, requiredSchema, new ArrayList<>(), cdcOffsetFilter);
+    }
+
+    @Override
+    protected SparkCellIterator buildCellIterator(int partitionId,
+                                                  @NotNull DataLayer dataLayer,
+                                                  @Nullable StructType columnFilter,
+                                                  @NotNull List<PartitionKeyFilter> partitionKeyFilters,
+                                                  @Nullable CdcOffsetFilter cdcOffsetFilter)
+    {
+        return new CdcCellIterator(partitionId, dataLayer, columnFilter, partitionKeyFilters, cdcOffsetFilter);
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/cdc/watermarker/InMemoryWatermarker.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/cdc/watermarker/InMemoryWatermarker.java
new file mode 100644
index 0000000..be29f66
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/cdc/watermarker/InMemoryWatermarker.java
@@ -0,0 +1,301 @@
+/*
+ * 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.cassandra.spark.cdc.watermarker;
+
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import javax.annotation.concurrent.ThreadSafe;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.spark.cdc.CommitLog;
+import org.apache.cassandra.spark.cdc.IPartitionUpdateWrapper;
+import org.apache.cassandra.spark.data.partitioner.CassandraInstance;
+import org.apache.spark.TaskContext;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * In-memory watermark implementation that caches position to start reading from each instance.
+ *
+ * WARNING: This implementation is for local testing only and should not be used in a Spark cluster.
+ *          The task allocation in Spark cannot guarantee a partition will be assigned to the same executor.
+ */
+@ThreadSafe
+public class InMemoryWatermarker implements Watermarker
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(InMemoryWatermarker.class);
+
+    public static final InMemoryWatermarker INSTANCE = new InMemoryWatermarker();
+
+    @VisibleForTesting
+    public static String TEST_THREAD_NAME = null;
+    // CHECKSTYLE IGNORE: Non-final to allow bypassing of TaskContext check by unit tests, this is still a constant
+
+    // Store watermarker per Spark job
+    protected final Map<String, JobWatermarker> jobs = new ConcurrentHashMap<>();
+
+    @Override
+    public Watermarker instance(String jobId)
+    {
+        return jobs.computeIfAbsent(jobId, this::newInstance).get();
+    }
+
+    @Override
+    public void recordReplicaCount(IPartitionUpdateWrapper update, int numReplicas)
+    {
+        throw new IllegalAccessError();
+    }
+
+    @Override
+    public int replicaCount(IPartitionUpdateWrapper update)
+    {
+        throw new IllegalAccessError();
+    }
+
+    // Allow sub-classes to override with own implementation
+    public JobWatermarker newInstance(String jobId)
+    {
+        return new JobWatermarker(jobId);
+    }
+
+    @Override
+    public void untrackReplicaCount(IPartitionUpdateWrapper update)
+    {
+        throw new IllegalAccessError();
+    }
+
+    @Override
+    public boolean seenBefore(IPartitionUpdateWrapper update)
+    {
+        throw new IllegalAccessError();
+    }
+
+    @Override
+    public void updateHighWaterMark(CommitLog.Marker marker)
+    {
+        throw new IllegalAccessError();
+    }
+
+    @Override
+    @Nullable
+    public CommitLog.Marker highWaterMark(CassandraInstance instance)
+    {
+        throw new IllegalAccessError();
+    }
+
+    @Override
+    public void persist(@Nullable Long maxAgeMicros)
+    {
+        throw new IllegalAccessError();
+    }
+
+    @Override
+    public void clear()
+    {
+        jobs.values().forEach(JobWatermarker::clear);
+        jobs.clear();
+    }
+
+    /**
+     * Stores per Spark partition watermarker for a given Spark job.
+     */
+    @ThreadSafe
+    public static class JobWatermarker implements Watermarker
+    {
+        protected final String jobId;
+        protected final Map<Integer, PartitionWatermarker> watermarkers = new ConcurrentHashMap<>();
+
+        public JobWatermarker(String jobId)
+        {
+            this.jobId = jobId;
+        }
+
+        @SuppressWarnings("unused")
+        public String jobId()
+        {
+            return jobId;
+        }
+
+        @Override
+        public Watermarker instance(String jobId)
+        {
+            Preconditions.checkArgument(this.jobId.equals(jobId));
+            return get();
+        }
+
+        @Override
+        public void recordReplicaCount(IPartitionUpdateWrapper update, int numReplicas)
+        {
+            get().recordReplicaCount(update, numReplicas);
+        }
+
+        @Override
+        public int replicaCount(IPartitionUpdateWrapper update)
+        {
+            return get().replicaCount(update);
+        }
+
+        @Override
+        public void untrackReplicaCount(IPartitionUpdateWrapper update)
+        {
+            get().untrackReplicaCount(update);
+        }
+
+        @Override
+        public boolean seenBefore(IPartitionUpdateWrapper update)
+        {
+            return get().seenBefore(update);
+        }
+
+        @Override
+        public void updateHighWaterMark(CommitLog.Marker marker)
+        {
+            get().updateHighWaterMark(marker);
+        }
+
+        @Override
+        public CommitLog.Marker highWaterMark(CassandraInstance instance)
+        {
+            return get().highWaterMark(instance);
+        }
+
+        @Override
+        public void persist(@Nullable Long maxAgeMicros)
+        {
+            get().persist(maxAgeMicros);
+        }
+
+        @Override
+        public void clear()
+        {
+            watermarkers.values().forEach(Watermarker::clear);
+            watermarkers.clear();
+        }
+
+        public PartitionWatermarker get()
+        {
+            if (!Thread.currentThread().getName().equals(TEST_THREAD_NAME))
+            {
+                Preconditions.checkNotNull(TaskContext.get(), "This method must be called by a Spark executor thread");
+            }
+            return watermarkers.computeIfAbsent(TaskContext.getPartitionId(), this::newInstance);
+        }
+
+        // Allow sub-classes to override with own implementation
+        public PartitionWatermarker newInstance(int partitionId)
+        {
+            return new PartitionWatermarker(partitionId);
+        }
+    }
+
+    /**
+     * Tracks highwater mark per instance and number of replicas previously received for updates that did not achieve the consistency level
+     */
+    public static class PartitionWatermarker implements Watermarker
+    {
+        // Tracks replica count for mutations with insufficient replica copies
+        protected final Map<IPartitionUpdateWrapper, Integer> replicaCount = new ConcurrentHashMap<>(1024);
+        // High watermark tracks how far we have read in the CommitLogs per CassandraInstance
+        protected final Map<CassandraInstance, CommitLog.Marker> highWatermarks = new ConcurrentHashMap<>();
+
+        int partitionId;
+
+        public PartitionWatermarker(int partitionId)
+        {
+            this.partitionId = partitionId;
+        }
+
+        public int partitionId()
+        {
+            return partitionId;
+        }
+
+        @Override
+        public Watermarker instance(String jobId)
+        {
+            return this;
+        }
+
+        @Override
+        public void recordReplicaCount(IPartitionUpdateWrapper update, int numReplicas)
+        {
+            replicaCount.put(update, numReplicas);
+        }
+
+        @Override
+        public int replicaCount(IPartitionUpdateWrapper update)
+        {
+            return replicaCount.getOrDefault(update, 0);
+        }
+
+        @Override
+        public void untrackReplicaCount(IPartitionUpdateWrapper update)
+        {
+            replicaCount.remove(update);
+        }
+
+        @Override
+        public boolean seenBefore(IPartitionUpdateWrapper update)
+        {
+            return replicaCount.containsKey(update);
+        }
+
+        @Override
+        public void updateHighWaterMark(CommitLog.Marker marker)
+        {
+            // This method will be called by executor thread when reading through CommitLog
+            // so use AtomicReference to ensure thread-safe and visible to other threads
+            if (marker == highWatermarks.merge(marker.instance(), marker,
+                    (oldValue, newValue) -> newValue.compareTo(oldValue) > 0 ? newValue : oldValue))
+            {
+                LOGGER.debug("Updated highwater mark instance={} marker='{}' partitionId={}",
+                             marker.instance().nodeName(), marker, partitionId());
+            }
+        }
+
+        @Override
+        public CommitLog.Marker highWaterMark(CassandraInstance instance)
+        {
+            return highWatermarks.get(instance);
+        }
+
+        @Override
+        public void persist(@Nullable Long maxAgeMicros)
+        {
+            replicaCount.keySet().removeIf(u -> isExpired(u, maxAgeMicros));
+        }
+
+        @Override
+        public void clear()
+        {
+            replicaCount.clear();
+            highWatermarks.clear();
+        }
+
+        public boolean isExpired(@NotNull IPartitionUpdateWrapper update, @Nullable Long maxAgeMicros)
+        {
+            return maxAgeMicros != null && update.maxTimestampMicros() < maxAgeMicros;
+        }
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/MD5Hash.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/MD5Hash.java
new file mode 100644
index 0000000..df8c29e
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/MD5Hash.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.cassandra.spark.common;
+
+import java.security.MessageDigest;
+import java.util.Base64;
+
+public final class MD5Hash
+{
+    private final String value;
+
+    private MD5Hash(MessageDigest digest)
+    {
+        value = Base64.getEncoder().encodeToString(digest.digest());
+    }
+
+    public static MD5Hash fromDigest(MessageDigest messageDigest)
+    {
+        return new MD5Hash(messageDigest);
+    }
+
+    public String toString()
+    {
+        return value;
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/SSTables.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/SSTables.java
new file mode 100644
index 0000000..9571cf6
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/SSTables.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.cassandra.spark.common;
+
+import java.nio.file.Path;
+
+public final class SSTables
+{
+    private SSTables()
+    {
+        throw new IllegalStateException(getClass() + " is static utility class and shall not be instantiated");
+    }
+
+    public static String getSSTableBaseName(Path dataFile)
+    {
+        String fileName = dataFile.getFileName().toString();
+        return fileName.substring(0, fileName.lastIndexOf("-") + 1);
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/client/ClientException.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/client/ClientException.java
new file mode 100644
index 0000000..4dea3f8
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/client/ClientException.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.cassandra.spark.common.client;
+
+import java.io.IOException;
+
+import org.jetbrains.annotations.Nullable;
+
+public class ClientException extends IOException
+{
+    public ClientException(@Nullable String message)
+    {
+        super(message);
+    }
+
+    public ClientException(@Nullable String message, @Nullable Throwable cause)
+    {
+        super(message, cause);
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/client/InstanceState.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/client/InstanceState.java
new file mode 100644
index 0000000..7485bd0
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/client/InstanceState.java
@@ -0,0 +1,33 @@
+/*
+ * 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.cassandra.spark.common.client;
+
+/**
+ * Valid instance states
+ */
+public enum InstanceState
+{
+    NORMAL,
+    LEAVING,
+    MOVING,
+    JOINING,
+    STARTING,
+    UNKNOWN
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/client/InstanceStatus.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/client/InstanceStatus.java
new file mode 100644
index 0000000..9cab0f7
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/client/InstanceStatus.java
@@ -0,0 +1,30 @@
+/*
+ * 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.cassandra.spark.common.client;
+
+/**
+ * Valid instance status
+ */
+public enum InstanceStatus
+{
+    UP,
+    DOWN,
+    UNKNOWN
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/model/BulkFeatures.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/model/BulkFeatures.java
new file mode 100644
index 0000000..c0c0726
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/model/BulkFeatures.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.cassandra.spark.common.model;
+
+@SuppressWarnings("unused")  // Part of this library's API used by the consumers
+public final class BulkFeatures
+{
+    public static final String BULK_WRITER = "cassandra-spark-bulk-writer";
+    public static final String BULK_READER = "cassandra-spark-bulk-reader";
+
+    private BulkFeatures()
+    {
+        throw new IllegalStateException(getClass() + " is static utility class and shall not be instantiated");
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/model/CassandraInstance.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/model/CassandraInstance.java
new file mode 100644
index 0000000..c0493fb
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/model/CassandraInstance.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.cassandra.spark.common.model;
+
+public interface CassandraInstance
+{
+    String getToken();
+
+    String getNodeName();
+
+    String getDataCenter();
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/BooleanType.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/BooleanType.java
new file mode 100644
index 0000000..8bfb555
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/BooleanType.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.cassandra.spark.common.schema;
+
+import java.nio.ByteBuffer;
+
+public class BooleanType implements ColumnType<Boolean>
+{
+    private Boolean getBoolean(ByteBuffer bytes)
+    {
+        byte value = bytes.get(bytes.position());
+        return value != 0;
+    }
+
+    @Override
+    public Boolean parseColumn(ByteBuffer buffer, int length)
+    {
+        byte[] value = new byte[length];
+        buffer.get(value, 0, length);
+        return getBoolean(ByteBuffer.wrap(value));
+    }
+
+    @Override
+    public Boolean getDefault()
+    {
+        return false;
+    }
+
+    @Override
+    public ByteBuffer serialize(Boolean value)
+    {
+        return ByteBuffer.allocate(1).put(0, (byte) (value ? 1 : 0));
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/BytesType.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/BytesType.java
new file mode 100644
index 0000000..e4b03b3
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/BytesType.java
@@ -0,0 +1,55 @@
+/*
+ * 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.cassandra.spark.common.schema;
+
+import java.nio.Buffer;
+import java.nio.ByteBuffer;
+
+public class BytesType implements ColumnType<ByteBuffer>
+{
+    @Override
+    public ByteBuffer parseColumn(ByteBuffer buffer, int length)
+    {
+        byte[] value = new byte[length];
+        buffer.get(value, 0, length);
+        return ByteBuffer.wrap(value);
+    }
+
+    @Override
+    public ByteBuffer getDefault()
+    {
+        return ByteBuffer.wrap(new byte[0]);
+    }
+
+    @Override
+    public ByteBuffer serialize(ByteBuffer value)
+    {
+        int position = value.position();
+        try
+        {
+            // Cast to ByteBuffer required when compiling with Java 8
+            return (ByteBuffer) ByteBuffer.allocate(value.remaining()).put(value).flip();
+        }
+        finally
+        {
+            ((Buffer) value).position(position);
+        }
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/CollectionType.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/CollectionType.java
new file mode 100644
index 0000000..a786ef3
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/CollectionType.java
@@ -0,0 +1,49 @@
+/*
+ * 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.cassandra.spark.common.schema;
+
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+public abstract class CollectionType<EntryType, IntermediateType> implements ColumnType<Collection<EntryType>>, Serializable
+{
+    @Override
+    public Collection<EntryType> parseColumn(ByteBuffer buffer, int length)
+    {
+        throw new UnsupportedOperationException("parseColumn() can't be called on CollectionType");
+    }
+
+    public abstract IntermediateType parseCollectionColumn(ByteBuffer colNameSuffix, ByteBuffer colValue);
+
+    public Collection<EntryType> getDefault()
+    {
+        return Collections.emptyList();
+    }
+
+    public abstract List<EntryType> finaliseCollection(List<IntermediateType> entryList);
+
+    public ByteBuffer serialize(Collection<EntryType> value)
+    {
+        throw new UnsupportedOperationException("Doesn't make much sense");
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/ColumnType.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/ColumnType.java
new file mode 100644
index 0000000..9a39c59
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/ColumnType.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.cassandra.spark.common.schema;
+
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+
+public interface ColumnType<T> extends Serializable
+{
+    /**
+     * Parses a value of this type from buffer. Value will be parsed from current position of the buffer. After
+     * completion of the function, position will be moved by "length" bytes.
+     *
+     * @param buffer Buffer to parse column from
+     * @param length Serialized value size in buffer is as big as length
+     * @return value as Java type
+     */
+    T parseColumn(ByteBuffer buffer, int length);
+
+    /**
+     * Default value, in case column value doesn't exist for CQL row
+     *
+     * @return the default value for the column type
+     */
+    T getDefault();
+
+    /**
+     * Serialize into ByteBuffer and keeps the position at beginning of ByteBuffer
+     *
+     * @param value the value to serialize
+     * @return A ByteBuffer containing the serialized value
+     */
+    ByteBuffer serialize(T value);
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/ColumnTypes.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/ColumnTypes.java
new file mode 100644
index 0000000..a6d35d8
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/ColumnTypes.java
@@ -0,0 +1,108 @@
+/*
+ * 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.cassandra.spark.common.schema;
+
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+
+@SuppressWarnings("unused")  // Extra types can become useful in the future
+public final class ColumnTypes implements Serializable
+{
+    // Non-Nullable Types
+    public static final ColumnType<String> STRING = new StringType();
+    public static final ColumnType<Integer> INT = new IntegerType();
+    public static final ColumnType<Long> LONG = new LongType();
+    public static final ColumnType<ByteBuffer> BYTES = new BytesType();
+    public static final ColumnType<String> STRING_UUID = new StringUuidType();
+    public static final ColumnType<java.util.UUID> UUID = new UuidType();
+    public static final ColumnType<Double> DOUBLE = new DoubleType();
+    public static final ColumnType<Boolean> BOOLEAN = new BooleanType();
+
+    // Nullable Types
+    public static final ColumnType<String> NULLABLE_STRING = new StringType()
+    {
+        @Override
+        public String getDefault()
+        {
+            return null;
+        }
+    };
+    public static final ColumnType<Integer> NULLABLE_INT = new IntegerType()
+    {
+        @Override
+        public Integer getDefault()
+        {
+            return null;
+        }
+    };
+    public static final ColumnType<Long> NULLABLE_LONG = new LongType()
+    {
+        @Override
+        public Long getDefault()
+        {
+            return null;
+        }
+    };
+    public static final ColumnType<ByteBuffer> NULLABLE_BYTES = new BytesType()
+    {
+        @Override
+        public ByteBuffer getDefault()
+        {
+            return null;
+        }
+    };
+    public static final ColumnType<String> NULLABLE_STRING_UUID = new StringUuidType()
+    {
+        @Override
+        public String getDefault()
+        {
+            return null;
+        }
+    };
+    public static final ColumnType<java.util.UUID> NULLABLE_UUID = new UuidType()
+    {
+        @Override
+        public java.util.UUID getDefault()
+        {
+            return null;
+        }
+    };
+    public static final ColumnType<Double> NULLABLE_DOUBLE = new DoubleType()
+    {
+        @Override
+        public Double getDefault()
+        {
+            return null;
+        }
+    };
+    public static final ColumnType<Boolean> NULLABLE_BOOLEAN = new BooleanType()
+    {
+        @Override
+        public Boolean getDefault()
+        {
+            return null;
+        }
+    };
+
+    private ColumnTypes()
+    {
+        throw new IllegalStateException(getClass() + " is static utility class and shall not be instantiated");
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/ColumnUtil.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/ColumnUtil.java
new file mode 100644
index 0000000..8de2674
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/ColumnUtil.java
@@ -0,0 +1,54 @@
+/*
+ * 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.cassandra.spark.common.schema;
+
+import java.nio.ByteBuffer;
+
+public final class ColumnUtil
+{
+    private ColumnUtil()
+    {
+        throw new IllegalStateException(getClass() + " is static utility class and shall not be instantiated");
+    }
+
+    private static int getLength(ByteBuffer buffer)
+    {
+        if (buffer.remaining() < 2)
+        {
+            return 0;
+        }
+        short length = buffer.getShort();
+        if (buffer.remaining() < length)
+        {
+            throw new IllegalArgumentException("Invalid buffer length, expecting " + length + " bytes but got " + buffer.remaining());
+        }
+        return length;
+    }
+
+    public static <T> T getField(ByteBuffer buffer, ColumnType<T> column)
+    {
+        return column.parseColumn(buffer, getLength(buffer));
+    }
+
+    public static <T> T parseSingleColumn(ColumnType<T> column, ByteBuffer buffer)
+    {
+        return column.parseColumn(buffer, buffer.remaining());
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/DoubleType.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/DoubleType.java
new file mode 100644
index 0000000..22b78bb
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/DoubleType.java
@@ -0,0 +1,50 @@
+/*
+ * 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.cassandra.spark.common.schema;
+
+import java.nio.ByteBuffer;
+
+public class DoubleType implements ColumnType<Double>
+{
+    private Double getDouble(ByteBuffer bytes)
+    {
+        return bytes.getDouble(bytes.position());
+    }
+
+    @Override
+    public Double parseColumn(ByteBuffer buffer, int length)
+    {
+        byte[] value = new byte[length];
+        buffer.get(value, 0, length);
+        return getDouble(ByteBuffer.wrap(value));
+    }
+
+    @Override
+    public Double getDefault()
+    {
+        return 0d;
+    }
+
+    @Override
+    public ByteBuffer serialize(Double value)
+    {
+        return ByteBuffer.allocate(8).putDouble(0, value);
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/IntegerType.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/IntegerType.java
new file mode 100644
index 0000000..5c7f113
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/IntegerType.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.cassandra.spark.common.schema;
+
+import java.nio.ByteBuffer;
+
+public class IntegerType implements ColumnType<Integer>
+{
+    @Override
+    public Integer parseColumn(ByteBuffer buffer, int length)
+    {
+        assert length == Integer.SIZE / Byte.SIZE;
+        return buffer.getInt();
+    }
+
+    @Override
+    public Integer getDefault()
+    {
+        return 0;
+    }
+
+    @Override
+    public ByteBuffer serialize(Integer value)
+    {
+        return ByteBuffer.allocate(Integer.SIZE / Byte.SIZE).putInt(0, value);
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/ListType.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/ListType.java
new file mode 100644
index 0000000..c3c15fd
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/ListType.java
@@ -0,0 +1,70 @@
+/*
+ * 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.cassandra.spark.common.schema;
+
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+
+import org.jetbrains.annotations.NotNull;
+
+public class ListType<T> extends CollectionType<T, ListType.CQLListEntry<T>>
+{
+    public final ColumnType<T> elementType;
+
+    public ListType(ColumnType<T> elementType)
+    {
+        this.elementType = elementType;
+    }
+
+    @Override
+    public CQLListEntry<T> parseCollectionColumn(ByteBuffer colNameSuffix, ByteBuffer colValue)
+    {
+        return new CQLListEntry<>(ColumnUtil.getField(colNameSuffix, ColumnTypes.UUID),
+                                  ColumnUtil.parseSingleColumn(elementType, colValue));
+    }
+
+    @Override
+    public List<T> finaliseCollection(List<CQLListEntry<T>> entryList)
+    {
+        Collections.sort(entryList);
+        return entryList.stream().map(entry -> entry.value).collect(Collectors.toList());
+    }
+
+    public static class CQLListEntry<T> implements Comparable<CQLListEntry<T>>
+    {
+        private final UUID timeUUID;
+        public final T value;
+
+        @Override
+        public int compareTo(@NotNull CQLListEntry that)
+        {
+            return this.timeUUID.compareTo(that.timeUUID);
+        }
+
+        CQLListEntry(UUID timeUUID, T value)
+        {
+            this.timeUUID = timeUUID;
+            this.value = value;
+        }
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/LongType.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/LongType.java
new file mode 100644
index 0000000..25ec619
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/LongType.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.cassandra.spark.common.schema;
+
+import java.nio.ByteBuffer;
+
+public class LongType implements ColumnType<Long>
+{
+    @Override
+    public Long parseColumn(ByteBuffer buffer, int length)
+    {
+        assert length == Long.SIZE / Byte.SIZE;
+        return buffer.getLong();
+    }
+
+    @Override
+    public Long getDefault()
+    {
+        return 0L;
+    }
+
+    @Override
+    public ByteBuffer serialize(Long value)
+    {
+        return ByteBuffer.allocate(Long.SIZE / Byte.SIZE).putLong(0, value);
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/MapType.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/MapType.java
new file mode 100644
index 0000000..5cadd51
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/MapType.java
@@ -0,0 +1,70 @@
+/*
+ * 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.cassandra.spark.common.schema;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+
+public class MapType<K, V> extends CollectionType<Map.Entry<K, V>, Map.Entry<K, V>>
+{
+    public final ColumnType<K> keyType;
+    public final ColumnType<V> valueType;
+
+    public MapType(ColumnType<K> keyType, ColumnType<V> valueType)
+    {
+        this.keyType = keyType;
+        this.valueType = valueType;
+    }
+
+    @Override
+    public Map.Entry<K, V> parseCollectionColumn(ByteBuffer colNameSuffix, ByteBuffer colValue)
+    {
+        K key = ColumnUtil.getField(colNameSuffix, keyType);
+        V value = ColumnUtil.parseSingleColumn(valueType, colValue);
+
+        return new Map.Entry<K, V>()
+        {
+            @Override
+            public K getKey()
+            {
+                return key;
+            }
+
+            @Override
+            public V getValue()
+            {
+                return value;
+            }
+
+            @Override
+            public V setValue(Object value)
+            {
+                throw new UnsupportedOperationException();
+            }
+        };
+    }
+
+    @Override
+    public List<Map.Entry<K, V>> finaliseCollection(List<Map.Entry<K, V>> entryList)
+    {
+        return entryList;
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/SetType.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/SetType.java
new file mode 100644
index 0000000..1b97e34
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/SetType.java
@@ -0,0 +1,45 @@
+/*
+ * 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.cassandra.spark.common.schema;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+public class SetType<T> extends CollectionType<T, T>
+{
+    public final ColumnType<T> elementType;
+
+    public SetType(ColumnType<T> elementType)
+    {
+        this.elementType = elementType;
+    }
+
+    @Override
+    public T parseCollectionColumn(ByteBuffer colNameSuffix, ByteBuffer colValue)
+    {
+        return ColumnUtil.getField(colNameSuffix, elementType);
+    }
+
+    @Override
+    public List<T> finaliseCollection(List<T> entryList)
+    {
+        return entryList;
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/StringType.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/StringType.java
new file mode 100644
index 0000000..9e2d228
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/StringType.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.cassandra.spark.common.schema;
+
+import java.nio.ByteBuffer;
+import java.nio.charset.Charset;
+
+public class StringType implements ColumnType<String>
+{
+    private static final Charset UTF_8 = Charset.forName("UTF-8");
+
+    @Override
+    public String parseColumn(ByteBuffer buffer, int length)
+    {
+        byte[] value = new byte[length];
+        buffer.get(value, 0, length);
+        return new String(value, UTF_8);
+    }
+
+    @Override
+    public String getDefault()
+    {
+        return "";
+    }
+
+    @Override
+    public ByteBuffer serialize(String value)
+    {
+        return ByteBuffer.wrap(value.getBytes(UTF_8));
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/StringUuidType.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/StringUuidType.java
new file mode 100644
index 0000000..9253cd6
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/StringUuidType.java
@@ -0,0 +1,45 @@
+/*
+ * 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.cassandra.spark.common.schema;
+
+import java.nio.ByteBuffer;
+import java.util.UUID;
+
+public class StringUuidType implements ColumnType<String>
+{
+    @Override
+    public String parseColumn(ByteBuffer buffer, int length)
+    {
+        return ColumnTypes.UUID.parseColumn(buffer, length).toString();
+    }
+
+    @Override
+    public String getDefault()
+    {
+        return "";
+    }
+
+    @Override
+    public ByteBuffer serialize(String valueStr)
+    {
+        UUID value = UUID.fromString(valueStr);
+        return ColumnTypes.UUID.serialize(value);
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/UuidType.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/UuidType.java
new file mode 100644
index 0000000..f6b4b82
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/common/schema/UuidType.java
@@ -0,0 +1,56 @@
+/*
+ * 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.cassandra.spark.common.schema;
+
+import java.nio.ByteBuffer;
+import java.util.UUID;
+
+public class UuidType implements ColumnType<UUID>
+{
+    private UUID getUUID(ByteBuffer raw)
+    {
+        return new UUID(raw.getLong(raw.position()), raw.getLong(raw.position() + 8));
+    }
+
+    private UUID parseUUID(ByteBuffer buffer, int length)
+    {
+        byte[] value = new byte[length];
+        buffer.get(value, 0, length);
+        return getUUID(ByteBuffer.wrap(value));
+    }
+
+    @Override
+    public UUID parseColumn(ByteBuffer buffer, int length)
+    {
+        return parseUUID(buffer, length);
+    }
+
+    @Override
+    public UUID getDefault()
+    {
+        return UUID.randomUUID();
+    }
+
+    @Override
+    public ByteBuffer serialize(UUID value)
+    {
+        return ByteBuffer.allocate(16).putLong(0, value.getMostSignificantBits()).putLong(8, value.getLeastSignificantBits());
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/BasicSupplier.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/BasicSupplier.java
new file mode 100644
index 0000000..0747b8d
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/BasicSupplier.java
@@ -0,0 +1,46 @@
+/*
+ * 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.cassandra.spark.data;
+
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.cassandra.spark.reader.SparkSSTableReader;
+import org.apache.cassandra.spark.utils.Throwing;
+import org.jetbrains.annotations.NotNull;
+
+public class BasicSupplier extends SSTablesSupplier
+{
+    private final Set<SSTable> ssTables;
+
+    public BasicSupplier(@NotNull Set<SSTable> ssTables)
+    {
+        this.ssTables = ssTables;
+    }
+
+    @Override
+    public <T extends SparkSSTableReader> Set<T> openAll(ReaderOpener<T> readerOpener)
+    {
+        return ssTables.stream()
+                       .map(Throwing.function(ssTable -> readerOpener.openReader(ssTable, true)))
+                       .filter(reader -> !reader.ignore())
+                       .collect(Collectors.toSet());
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/CassandraDataLayer.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/CassandraDataLayer.java
new file mode 100644
index 0000000..259e468
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/CassandraDataLayer.java
@@ -0,0 +1,1169 @@
+/*
+ * 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.cassandra.spark.data;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.Serializable;
+import java.math.BigInteger;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionStage;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Range;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+import org.apache.cassandra.bridge.BigNumberConfig;
+import org.apache.cassandra.bridge.BigNumberConfigImpl;
+import org.apache.cassandra.bridge.CassandraBridge;
+import org.apache.cassandra.bridge.CassandraBridgeFactory;
+import org.apache.cassandra.bridge.CassandraVersion;
+import org.apache.cassandra.clients.ExecutorHolder;
+import org.apache.cassandra.clients.Sidecar;
+import org.apache.cassandra.clients.SidecarInstanceImpl;
+import org.apache.cassandra.clients.SslConfig;
+import org.apache.cassandra.sidecar.client.SidecarClient;
+import org.apache.cassandra.sidecar.client.SidecarInstance;
+import org.apache.cassandra.sidecar.client.SimpleSidecarInstancesProvider;
+import org.apache.cassandra.sidecar.client.exception.RetriesExhaustedException;
+import org.apache.cassandra.sidecar.common.NodeSettings;
+import org.apache.cassandra.sidecar.common.data.ListSnapshotFilesResponse;
+import org.apache.cassandra.sidecar.common.data.RingResponse;
+import org.apache.cassandra.sidecar.common.data.SchemaResponse;
+import org.apache.cassandra.spark.cdc.CommitLog;
+import org.apache.cassandra.spark.cdc.TableIdLookup;
+import org.apache.cassandra.spark.cdc.watermarker.Watermarker;
+import org.apache.cassandra.spark.config.SchemaFeature;
+import org.apache.cassandra.spark.config.SchemaFeatureSet;
+import org.apache.cassandra.spark.data.partitioner.CassandraInstance;
+import org.apache.cassandra.spark.data.partitioner.CassandraRing;
+import org.apache.cassandra.spark.data.partitioner.ConsistencyLevel;
+import org.apache.cassandra.spark.data.partitioner.Partitioner;
+import org.apache.cassandra.spark.data.partitioner.TokenPartitioner;
+import org.apache.cassandra.spark.sparksql.LastModifiedTimestampDecorator;
+import org.apache.cassandra.spark.sparksql.RowBuilder;
+import org.apache.cassandra.spark.stats.Stats;
+import org.apache.cassandra.spark.utils.CqlUtils;
+import org.apache.cassandra.spark.utils.MapUtils;
+import org.apache.cassandra.spark.utils.ScalaFunctions;
+import org.apache.cassandra.spark.utils.ThrowableUtils;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.util.ShutdownHookManager;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.cassandra.spark.utils.Properties.NODE_STATUS_NOT_CONSIDERED;
+
+public class CassandraDataLayer extends PartitionedDataLayer implements Serializable
+{
+    private static final long serialVersionUID = -9038926850642710787L;
+
+    public static final Logger LOGGER = LoggerFactory.getLogger(CassandraDataLayer.class);
+    private static final Cache<String, CompletableFuture<List<SSTable>>> SNAPSHOT_CACHE =
+            CacheBuilder.newBuilder()
+                        .expireAfterAccess(15, TimeUnit.MINUTES)
+                        .maximumSize(128)
+                        .build();
+
+    protected String snapshotName;
+    protected String keyspace;
+    protected String table;
+    protected CassandraBridge bridge;
+    protected Set<? extends SidecarInstance> clusterConfig;
+    protected TokenPartitioner tokenPartitioner;
+    protected Map<String, AvailabilityHint> availabilityHints;
+    protected Sidecar.ClientConfig sidecarClientConfig;
+    private SslConfig sslConfig;
+    protected Map<String, BigNumberConfigImpl> bigNumberConfigMap;
+    protected boolean enableStats;
+    protected boolean readIndexOffset;
+    protected boolean useIncrementalRepair;
+    protected List<SchemaFeature> requestedFeatures;
+    protected Map<String, ReplicationFactor> rfMap;
+    @Nullable
+    protected String lastModifiedTimestampField;
+    // volatile in order to publish the reference for visibility
+    protected volatile CqlTable cqlTable;
+    protected transient SidecarClient sidecar;
+    @VisibleForTesting
+    transient Map<String, SidecarInstance> instanceMap;
+
+    public CassandraDataLayer(@NotNull ClientConfig options,
+                              @NotNull Sidecar.ClientConfig sidecarClientConfig,
+                              @Nullable SslConfig sslConfig)
+    {
+        super(options.consistencyLevel(), options.datacenter());
+        this.snapshotName = options.snapshotName();
+        this.keyspace = options.keyspace();
+        this.table = CqlUtils.cleanTableName(options.table());
+        this.sidecarClientConfig = sidecarClientConfig;
+        this.sslConfig = sslConfig;
+        this.bigNumberConfigMap = options.bigNumberConfigMap();
+        this.enableStats = options.enableStats();
+        this.readIndexOffset = options.readIndexOffset();
+        this.useIncrementalRepair = options.useIncrementalRepair();
+        this.lastModifiedTimestampField = options.lastModifiedTimestampField();
+        this.requestedFeatures = options.requestedFeatures();
+    }
+
+    // For serialization
+    @VisibleForTesting
+    // CHECKSTYLE IGNORE: Constructor with many parameters
+    CassandraDataLayer(@Nullable String keyspace,
+                       @Nullable String table,
+                       @NotNull String snapshotName,
+                       @Nullable String datacenter,
+                       @NotNull Sidecar.ClientConfig sidecarClientConfig,
+                       @Nullable SslConfig sslConfig,
+                       @NotNull CqlTable cqlTable,
+                       @NotNull TokenPartitioner tokenPartitioner,
+                       @NotNull CassandraVersion version,
+                       @NotNull ConsistencyLevel consistencyLevel,
+                       @NotNull Set<SidecarInstanceImpl> clusterConfig,
+                       @NotNull Map<String, PartitionedDataLayer.AvailabilityHint> availabilityHints,
+                       @NotNull Map<String, BigNumberConfigImpl> bigNumberConfigMap,
+                       boolean enableStats,
+                       boolean readIndexOffset,
+                       boolean useIncrementalRepair,
+                       @Nullable String lastModifiedTimestampField,
+                       List<SchemaFeature> requestedFeatures,
+                       @NotNull Map<String, ReplicationFactor> rfMap)
+    {
+        super(consistencyLevel, datacenter);
+        this.snapshotName = snapshotName;
+        this.keyspace = keyspace;
+        this.table = table;
+        this.cqlTable = cqlTable;
+        this.tokenPartitioner = tokenPartitioner;
+        this.bridge = CassandraBridgeFactory.get(version);
+        this.clusterConfig = clusterConfig;
+        this.availabilityHints = availabilityHints;
+        this.sidecarClientConfig = sidecarClientConfig;
+        this.sslConfig = sslConfig;
+        this.bigNumberConfigMap = bigNumberConfigMap;
+        this.enableStats = enableStats;
+        this.readIndexOffset = readIndexOffset;
+        this.useIncrementalRepair = useIncrementalRepair;
+        this.lastModifiedTimestampField = lastModifiedTimestampField;
+        this.requestedFeatures = requestedFeatures;
+        if (lastModifiedTimestampField != null)
+        {
+            aliasLastModifiedTimestamp(this.requestedFeatures, this.lastModifiedTimestampField);
+        }
+        this.rfMap = rfMap;
+        initInstanceMap();
+    }
+
+    public void initialize(@NotNull ClientConfig options)
+    {
+        dialHome(options);
+
+        LOGGER.info("Starting Cassandra Spark job snapshotName={} keyspace={} table={} dc={}",
+                    snapshotName, keyspace, table, datacenter);
+
+        // Load cluster config from Discovery
+        clusterConfig = initializeClusterConfig(options);
+        initInstanceMap();
+
+        // Get cluster info from CassandraManager
+        int effectiveNumberOfCores;
+        CompletableFuture<RingResponse> ringFuture = sidecar.ring(keyspace);
+        try
+        {
+            CompletableFuture<NodeSettings> nodeSettingsFuture = nodeSettingsFuture(clusterConfig, ringFuture);
+            effectiveNumberOfCores = initBulkReader(options, nodeSettingsFuture, ringFuture);
+        }
+        catch (InterruptedException exception)
+        {
+            Thread.currentThread().interrupt();
+            throw new RuntimeException(exception);
+        }
+        catch (ExecutionException exception)
+        {
+            throw new RuntimeException(ThrowableUtils.rootCause(exception));
+        }
+        LOGGER.info("Initialized Cassandra Bulk Reader with effectiveNumberOfCores={}", effectiveNumberOfCores);
+    }
+
+    private int initBulkReader(@NotNull ClientConfig options,
+                               CompletableFuture<NodeSettings> nodeSettingsFuture,
+                               CompletableFuture<RingResponse> ringFuture) throws ExecutionException, InterruptedException
+    {
+        Preconditions.checkArgument(keyspace != null, "Keyspace must be non-null for Cassandra Bulk Reader");
+        Preconditions.checkArgument(table != null, "Table must be non-null for Cassandra Bulk Reader");
+        CompletableFuture<Map<String, PartitionedDataLayer.AvailabilityHint>> snapshotFuture;
+        if (options.createSnapshot())
+        {
+            // Use create snapshot request to capture instance availability hint
+            LOGGER.info("Creating snapshot snapshotName={} keyspace={} table={} dc={}",
+                        snapshotName, keyspace, table, datacenter);
+            snapshotFuture = ringFuture.thenCompose(this::createSnapshot);
+        }
+        else
+        {
+            snapshotFuture = CompletableFuture.completedFuture(new HashMap<>());
+        }
+        ShutdownHookManager.addShutdownHook(org.apache.spark.util.ShutdownHookManager.TEMP_DIR_SHUTDOWN_PRIORITY(),
+                                            ScalaFunctions.wrapLambda(() -> shutdownHook(options)));
+
+        CompletableFuture<SchemaResponse> schemaFuture = sidecar.schema(keyspace);
+        NodeSettings nodeSettings = nodeSettingsFuture.get();
+
+        String cassandraVersion = getEffectiveCassandraVersionForRead(clusterConfig, nodeSettings);
+
+        Partitioner partitioner = Partitioner.from(nodeSettings.partitioner());
+        bridge = CassandraBridgeFactory.get(cassandraVersion);
+        availabilityHints = snapshotFuture.get();
+
+        String fullSchema = schemaFuture.get().schema();
+        String createStmt = CqlUtils.extractTableSchema(fullSchema, keyspace, table);
+        int indexCount = CqlUtils.extractIndexCount(fullSchema, keyspace, table);
+        Set<String> udts = CqlUtils.extractUdts(fullSchema, keyspace);
+        ReplicationFactor replicationFactor = CqlUtils.extractReplicationFactor(fullSchema, keyspace);
+        rfMap = ImmutableMap.of(keyspace, replicationFactor);
+        CompletableFuture<Integer> sizingFuture = CompletableFuture.supplyAsync(
+                () -> getSizing(clusterConfig, replicationFactor, options).getEffectiveNumberOfCores(),
+                ExecutorHolder.EXECUTOR_SERVICE);
+        validateReplicationFactor(replicationFactor);
+        udts.forEach(udt -> LOGGER.info("Adding schema UDT: '{}'", udt));
+
+        cqlTable = bridge().buildSchema(createStmt, keyspace, replicationFactor, partitioner, udts, null, indexCount);
+        CassandraRing ring = createCassandraRingFromRing(partitioner, replicationFactor, ringFuture.get());
+
+        int effectiveNumberOfCores = sizingFuture.get();
+        tokenPartitioner = new TokenPartitioner(ring, options.defaultParallelism, effectiveNumberOfCores);
+        return effectiveNumberOfCores;
+    }
+
+    protected void shutdownHook(ClientConfig options)
+    {
+        // Preserves previous behavior, but we may just want to check for the clearSnapshot option in the future
+        if (options.clearSnapshot())
+        {
+            if (options.createSnapshot())
+            {
+                clearSnapshot(clusterConfig, options);
+            }
+            else
+            {
+                LOGGER.warn("Skipping clearing snapshot because it was not created by this job. "
+                          + "Only the job that created the snapshot can clear it. "
+                          + "snapshotName={} keyspace={} table={} dc={}",
+                            snapshotName, keyspace, table, datacenter);
+            }
+        }
+
+        try
+        {
+            sidecar.close();
+        }
+        catch (Exception exception)
+        {
+            LOGGER.warn("Unable to close Sidecar", exception);
+        }
+    }
+
+    private CompletionStage<Map<String, AvailabilityHint>> createSnapshot(RingResponse ring)
+    {
+        Map<String, PartitionedDataLayer.AvailabilityHint> availabilityHints = new ConcurrentHashMap<>(ring.size());
+
+        // Fire off create snapshot request across the entire cluster
+        List<CompletableFuture<Void>> futures = ring
+                .stream()
+                .filter(ringEntry -> datacenter == null || datacenter.equals(ringEntry.datacenter()))
+                .map(ringEntry -> {
+                    PartitionedDataLayer.AvailabilityHint hint =
+                            PartitionedDataLayer.AvailabilityHint.fromState(ringEntry.status(), ringEntry.state());
+
+                    CompletableFuture<PartitionedDataLayer.AvailabilityHint> createSnapshotFuture;
+                    if (NODE_STATUS_NOT_CONSIDERED.contains(ringEntry.state()))
+                    {
+                        LOGGER.warn("Skip snapshot creating when node is joining or down "
+                                  + "snapshotName={} keyspace={} table={} datacenter={} fqdn={} status={} state={}",
+                                    snapshotName, keyspace, table, datacenter, ringEntry.fqdn(), ringEntry.status(), ringEntry.state());
+                        createSnapshotFuture = CompletableFuture.completedFuture(hint);
+                    }
+                    else
+                    {
+                        LOGGER.info("Creating snapshot on instance snapshotName={} keyspace={} table={} datacenter={} fqdn={}",
+                                    snapshotName, keyspace, table, datacenter, ringEntry.fqdn());
+                        SidecarInstance sidecarInstance = new SidecarInstanceImpl(ringEntry.fqdn(), sidecarClientConfig.port());
+                        createSnapshotFuture = sidecar
+                                .createSnapshot(sidecarInstance, keyspace, table, snapshotName)
+                                .handle((resp, throwable) -> {
+                                    if (throwable == null)
+                                    {
+                                        // Create snapshot succeeded
+                                        return hint;
+                                    }
+
+                                    if (isExhausted(throwable))
+                                    {
+                                        LOGGER.warn("Failed to create snapshot on instance", throwable);
+                                        return PartitionedDataLayer.AvailabilityHint.DOWN;
+                                    }
+
+                                    LOGGER.error("Unexpected error creating snapshot on instance", throwable);
+                                    return PartitionedDataLayer.AvailabilityHint.UNKNOWN;
+                                });
+                    }
+
+                    return createSnapshotFuture
+                           .thenAccept(h -> availabilityHints.put(ringEntry.fqdn(), h));
+                })
+                .collect(Collectors.toList());
+
+        return CompletableFuture
+               .allOf(futures.toArray(new CompletableFuture[0]))
+               .handle((results, throwable) -> availabilityHints);
+    }
+
+    private boolean isExhausted(@Nullable Throwable throwable)
+    {
+        return throwable != null && (throwable instanceof RetriesExhaustedException || isExhausted(throwable.getCause()));
+    }
+
+    @Override
+    public boolean useIncrementalRepair()
+    {
+        return useIncrementalRepair;
+    }
+
+    @Override
+    public boolean readIndexOffset()
+    {
+        return readIndexOffset;
+    }
+
+    protected void initInstanceMap()
+    {
+        instanceMap = clusterConfig.stream().collect(Collectors.toMap(SidecarInstance::hostname, Function.identity()));
+        try
+        {
+            sidecar = Sidecar.from(new SimpleSidecarInstancesProvider(new ArrayList<>(clusterConfig)),
+                                   sidecarClientConfig,
+                                   sslConfig);
+        }
+        catch (IOException ioException)
+        {
+            throw new RuntimeException("Unable to build sidecar client", ioException);
+        }
+        LOGGER.info("Initialized CassandraDataLayer instanceMap numInstances={}", instanceMap.size());
+    }
+
+    @Override
+    public CassandraBridge bridge()
+    {
+        return bridge;
+    }
+
+    @Override
+    public Stats stats()
+    {
+        return Stats.DoNothingStats.INSTANCE;
+    }
+
+    @Override
+    public List<SchemaFeature> requestedFeatures()
+    {
+        return requestedFeatures;
+    }
+
+    @Override
+    public CassandraRing ring()
+    {
+        return tokenPartitioner.ring();
+    }
+
+    @Override
+    public TokenPartitioner tokenPartitioner()
+    {
+        return tokenPartitioner;
+    }
+
+    @Override
+    protected ExecutorService executorService()
+    {
+        return ExecutorHolder.EXECUTOR_SERVICE;
+    }
+
+    @Override
+    public String jobId()
+    {
+        return null;
+    }
+
+    @Override
+    public CqlTable cqlTable()
+    {
+        if (cqlTable == null)
+        {
+            throw new RuntimeException("Schema not initialized");
+        }
+        return cqlTable;
+    }
+
+    @Override
+    public Watermarker cdcWatermarker()
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public Duration cdcWatermarkWindow()
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public CompletableFuture<List<CommitLog>> listCommitLogs(CassandraInstance instance)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public ReplicationFactor replicationFactor(String keyspace)
+    {
+        return rfMap.get(keyspace);
+    }
+
+    @Override
+    public TableIdLookup tableIdLookup()
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    protected PartitionedDataLayer.AvailabilityHint getAvailability(CassandraInstance instance)
+    {
+        // Hint CassandraInstance availability to parent PartitionedDataLayer
+        PartitionedDataLayer.AvailabilityHint hint = availabilityHints.get(instance.nodeName());
+        return hint != null ? hint : PartitionedDataLayer.AvailabilityHint.UNKNOWN;
+    }
+
+    private String snapshotKey(SidecarInstance instance)
+    {
+        return String.format("%s/%s/%d/%s/%s/%s",
+                             datacenter, instance.hostname(), instance.port(), keyspace, table, snapshotName);
+    }
+
+    @Override
+    public CompletableFuture<Stream<SSTable>> listInstance(int partitionId,
+                                                           @NotNull Range<BigInteger> range,
+                                                           @NotNull CassandraInstance instance)
+    {
+        SidecarInstance sidecarInstance = instanceMap.get(instance.nodeName());
+        if (sidecarInstance == null)
+        {
+            throw new IllegalStateException("Could not find matching cassandra instance: " + instance.nodeName());
+        }
+        String key = snapshotKey(sidecarInstance);  // NOTE: We don't currently support token filtering in list snapshot
+        LOGGER.info("Listing snapshot partition={} lowerBound={} upperBound={} "
+                  + "instance={} port={} keyspace={} tableName={} snapshotName={}",
+                    partitionId, range.lowerEndpoint(), range.upperEndpoint(),
+                    sidecarInstance.hostname(), sidecarInstance.port(), keyspace, table, snapshotName);
+        try
+        {
+            return SNAPSHOT_CACHE.get(key, () -> {
+                LOGGER.info("Listing instance snapshot partition={} lowerBound={} upperBound={} "
+                          + "instance={} port={} keyspace={} tableName={} snapshotName={} cacheKey={}",
+                            partitionId, range.lowerEndpoint(), range.upperEndpoint(),
+                            sidecarInstance.hostname(), sidecarInstance.port(), keyspace, table, snapshotName, key);
+                return sidecar.listSnapshotFiles(sidecarInstance, keyspace, table, snapshotName)
+                              .thenApply(response -> collectSSTableList(sidecarInstance, response, partitionId));
+            }).thenApply(Collection::stream);
+        }
+        catch (ExecutionException exception)
+        {
+            CompletableFuture<Stream<SSTable>> future = new CompletableFuture<>();
+            future.completeExceptionally(ThrowableUtils.rootCause(exception));
+            return future;
+        }
+    }
+
+    @SuppressWarnings("UnstableApiUsage")
+    private List<SSTable> collectSSTableList(SidecarInstance sidecarInstance,
+                                             ListSnapshotFilesResponse response,
+                                             int partitionId)
+    {
+        if (response == null)
+        {
+            throw new IncompleteSSTableException();
+        }
+        List<ListSnapshotFilesResponse.FileInfo> snapshotFilesInfo = response.snapshotFilesInfo();
+        if (snapshotFilesInfo == null)
+        {
+            throw new IncompleteSSTableException();
+        }
+
+        // Group SSTable components together
+        Map<String, Map<FileType, ListSnapshotFilesResponse.FileInfo>> result = new LinkedHashMap<>(1024);
+        for (ListSnapshotFilesResponse.FileInfo file : snapshotFilesInfo)
+        {
+            String fileName = file.fileName;
+            int lastIndexOfDash = fileName.lastIndexOf('-');
+            if (lastIndexOfDash < 0)
+            {
+                // E.g. dd manifest.json file
+                continue;
+            }
+            String ssTableName = fileName.substring(0, lastIndexOfDash);
+            try
+            {
+                FileType fileType = FileType.fromExtension(fileName.substring(lastIndexOfDash + 1));
+                result.computeIfAbsent(ssTableName, k -> new LinkedHashMap<>())
+                      .put(fileType, file);
+            }
+            catch (IllegalArgumentException ignore)
+            {
+                // Ignore unknown SSTable component types
+            }
+        }
+
+        // Map to SSTable
+        return result.values().stream()
+                              .map(components -> new SidecarProvisionedSSTable(sidecar,
+                                                                               sidecarClientConfig,
+                                                                               sidecarInstance,
+                                                                               keyspace,
+                                                                               table,
+                                                                               snapshotName,
+                                                                               components,
+                                                                               partitionId,
+                                                                               stats()))
+                              .collect(Collectors.toList());
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Objects.hash(super.hashCode(), cqlTable, snapshotName, keyspace, table, version());
+    }
+
+    @Override
+    public boolean equals(Object other)
+    {
+        if (this == other)
+        {
+            return true;
+        }
+        if (other == null || this.getClass() != other.getClass() || !super.equals(other))
+        {
+            return false;
+        }
+
+        CassandraDataLayer that = (CassandraDataLayer) other;
+        return cqlTable.equals(that.cqlTable)
+            && snapshotName.equals(that.snapshotName)
+            && keyspace.equals(that.keyspace)
+            && table.equals(that.table)
+            && version().equals(that.version());
+    }
+
+    public Map<String, BigNumberConfigImpl> bigNumberConfigMap()
+    {
+        return bigNumberConfigMap;
+    }
+
+    @Override
+    public BigNumberConfig bigNumberConfig(CqlField field)
+    {
+        BigNumberConfigImpl config = bigNumberConfigMap.get(field.name());
+        return config != null ? config : BigNumberConfig.DEFAULT;
+    }
+
+    /* Internal Cassandra SSTable */
+
+    @VisibleForTesting
+    public CassandraRing createCassandraRingFromRing(Partitioner partitioner,
+                                                     ReplicationFactor replicationFactor,
+                                                     RingResponse ring)
+    {
+        Collection<CassandraInstance> instances = ring
+                .stream()
+                .map(status -> new CassandraInstance(status.token(), status.fqdn(), status.datacenter()))
+                .collect(Collectors.toList());
+        return new CassandraRing(partitioner, keyspace, replicationFactor, instances);
+    }
+
+    // JDK Serialization
+
+    @SuppressWarnings("unchecked")
+    private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException
+    {
+        LOGGER.warn("Falling back to JDK deserialization");
+        this.bridge = CassandraBridgeFactory.get(CassandraVersion.valueOf(in.readUTF()));
+        this.snapshotName = in.readUTF();
+        this.keyspace = readNullable(in);
+        this.table = readNullable(in);
+        this.sidecarClientConfig = Sidecar.ClientConfig.create(in.readInt(),
+                                                               in.readInt(),
+                                                               in.readLong(),
+                                                               in.readLong(),
+                                                               in.readLong(),
+                                                               in.readLong(),
+                                                               in.readInt(),
+                                                               in.readInt(),
+                                                               (Map<FileType, Long>) in.readObject(),
+                                                               (Map<FileType, Long>) in.readObject());
+        this.sslConfig = (SslConfig) in.readObject();
+
+        this.cqlTable = bridge.javaDeserialize(in, CqlTable.class);  // Delegate (de-)serialization of version-specific objects to the Cassandra Bridge
+        this.tokenPartitioner = (TokenPartitioner) in.readObject();
+        this.clusterConfig = (Set<SidecarInstanceImpl>) in.readObject();
+        this.availabilityHints = (Map<String, AvailabilityHint>) in.readObject();
+        this.bigNumberConfigMap = (Map<String, BigNumberConfigImpl>) in.readObject();
+        this.enableStats = in.readBoolean();
+        this.readIndexOffset = in.readBoolean();
+        this.useIncrementalRepair = in.readBoolean();
+        this.lastModifiedTimestampField = readNullable(in);
+        int features = in.readShort();
+        List<SchemaFeature> requestedFeatures = new ArrayList<>(features);
+        for (int feature = 0; feature < features; feature++)
+        {
+            String featureName = in.readUTF();
+            requestedFeatures.add(SchemaFeatureSet.valueOf(featureName.toUpperCase()));
+        }
+        this.requestedFeatures = requestedFeatures;
+        // Has alias for last modified timestamp
+        if (this.lastModifiedTimestampField != null)
+        {
+            aliasLastModifiedTimestamp(this.requestedFeatures, this.lastModifiedTimestampField);
+        }
+        this.rfMap = (Map<String, ReplicationFactor>) in.readObject();
+        this.initInstanceMap();
+    }
+
+    private void writeObject(ObjectOutputStream out) throws IOException, ClassNotFoundException
+    {
+        LOGGER.warn("Falling back to JDK serialization");
+        out.writeUTF(this.version().name());
+        out.writeUTF(this.snapshotName);
+        writeNullable(out, this.keyspace);
+        writeNullable(out, this.table);
+        out.writeInt(this.sidecarClientConfig.port());
+        out.writeInt(this.sidecarClientConfig.maxRetries());
+        out.writeLong(this.sidecarClientConfig.millisToSleep());
+        out.writeLong(this.sidecarClientConfig.maxMillisToSleep());
+        out.writeLong(this.sidecarClientConfig.maxBufferSize());
+        out.writeLong(this.sidecarClientConfig.chunkBufferSize());
+        out.writeInt(this.sidecarClientConfig.maxPoolSize());
+        out.writeInt(this.sidecarClientConfig.timeoutSeconds());
+        out.writeObject(this.sidecarClientConfig.maxBufferOverride());
+        out.writeObject(this.sidecarClientConfig.chunkBufferOverride());
+        out.writeObject(this.sslConfig);
+        bridge.javaSerialize(out, this.cqlTable);  // Delegate (de-)serialization of version-specific objects to the Cassandra Bridge
+        out.writeObject(this.tokenPartitioner);
+        out.writeObject(this.clusterConfig);
+        out.writeObject(this.availabilityHints);
+        out.writeObject(this.bigNumberConfigMap);
+        out.writeBoolean(this.enableStats);
+        out.writeBoolean(this.readIndexOffset);
+        out.writeBoolean(this.useIncrementalRepair);
+        // If lastModifiedTimestampField exist, it aliases the LMT field
+        writeNullable(out, this.lastModifiedTimestampField);
+        // Write the list of requested features: first write the size, then write the feature names
+        out.writeShort(this.requestedFeatures.size());
+        for (SchemaFeature feature : requestedFeatures)
+        {
+            out.writeUTF(feature.optionName());
+        }
+        out.writeObject(this.rfMap);
+    }
+
+    private static void writeNullable(ObjectOutputStream out, @Nullable String string) throws IOException
+    {
+        if (string == null)
+        {
+            out.writeBoolean(false);
+        }
+        else
+        {
+            out.writeBoolean(true);
+            out.writeUTF(string);
+        }
+    }
+
+    @Nullable
+    private static String readNullable(ObjectInputStream in) throws IOException
+    {
+        if (in.readBoolean())
+        {
+            return in.readUTF();
+        }
+        return null;
+    }
+
+    // Kryo Serialization
+
+    public static class Serializer extends com.esotericsoftware.kryo.Serializer<CassandraDataLayer>
+    {
+        @Override
+        public void write(Kryo kryo, Output out, CassandraDataLayer dataLayer)
+        {
+            LOGGER.info("Serializing CassandraDataLayer with Kryo");
+            out.writeString(dataLayer.keyspace);
+            out.writeString(dataLayer.table);
+            out.writeString(dataLayer.snapshotName);
+            out.writeString(dataLayer.datacenter);
+            out.writeInt(dataLayer.sidecarClientConfig.port());
+            out.writeInt(dataLayer.sidecarClientConfig.maxRetries());
+            out.writeLong(dataLayer.sidecarClientConfig.millisToSleep());
+            out.writeLong(dataLayer.sidecarClientConfig.maxMillisToSleep());
+            out.writeLong(dataLayer.sidecarClientConfig.maxBufferSize());
+            out.writeLong(dataLayer.sidecarClientConfig.chunkBufferSize());
+            out.writeInt(dataLayer.sidecarClientConfig.maxPoolSize());
+            out.writeInt(dataLayer.sidecarClientConfig.timeoutSeconds());
+            kryo.writeObject(out, dataLayer.sidecarClientConfig.maxBufferOverride());
+            kryo.writeObject(out, dataLayer.sidecarClientConfig.chunkBufferOverride());
+            kryo.writeObjectOrNull(out, dataLayer.sslConfig, SslConfig.class);
+            kryo.writeObject(out, dataLayer.cqlTable);
+            kryo.writeObject(out, dataLayer.tokenPartitioner);
+            kryo.writeObject(out, dataLayer.version());
+            kryo.writeObject(out, dataLayer.consistencyLevel);
+            kryo.writeObject(out, dataLayer.clusterConfig);
+            kryo.writeObject(out, dataLayer.availabilityHints);
+            out.writeBoolean(dataLayer.bigNumberConfigMap.isEmpty());  // Kryo fails to deserialize bigNumberConfigMap map if empty
+            if (!dataLayer.bigNumberConfigMap.isEmpty())
+            {
+                kryo.writeObject(out, dataLayer.bigNumberConfigMap);
+            }
+            out.writeBoolean(dataLayer.enableStats);
+            out.writeBoolean(dataLayer.readIndexOffset);
+            out.writeBoolean(dataLayer.useIncrementalRepair);
+            // If lastModifiedTimestampField exist, it aliases the LMT field
+            out.writeString(dataLayer.lastModifiedTimestampField);
+            // Write the list of requested features: first write the size, then write the feature names
+            SchemaFeaturesListWrapper listWrapper = new SchemaFeaturesListWrapper();
+            listWrapper.requestedFeatureNames = dataLayer.requestedFeatures.stream()
+                                                                           .map(SchemaFeature::optionName)
+                                                                           .collect(Collectors.toList());
+            kryo.writeObject(out, listWrapper);
+            kryo.writeObject(out, dataLayer.rfMap);
+        }
+
+        @SuppressWarnings("unchecked")
+        @Override
+        public CassandraDataLayer read(Kryo kryo, Input in, Class<CassandraDataLayer> type)
+        {
+            LOGGER.info("Deserializing CassandraDataLayer with Kryo");
+            return new CassandraDataLayer(
+                    in.readString(),
+                    in.readString(),
+                    in.readString(),
+                    in.readString(),
+                    Sidecar.ClientConfig.create(in.readInt(),
+                                                in.readInt(),
+                                                in.readLong(),
+                                                in.readLong(),
+                                                in.readLong(),
+                                                in.readLong(),
+                                                in.readInt(),
+                                                in.readInt(),
+                                                (Map<FileType, Long>) kryo.readObject(in, HashMap.class),
+                                                (Map<FileType, Long>) kryo.readObject(in, HashMap.class)),
+                    kryo.readObjectOrNull(in, SslConfig.class),
+                    kryo.readObject(in, CqlTable.class),
+                    kryo.readObject(in, TokenPartitioner.class),
+                    kryo.readObject(in, CassandraVersion.class),
+                    kryo.readObject(in, ConsistencyLevel.class),
+                    kryo.readObject(in, HashSet.class),
+                    (Map<String, PartitionedDataLayer.AvailabilityHint>) kryo.readObject(in, HashMap.class),
+                    in.readBoolean() ? Collections.emptyMap()
+                                     : (Map<String, BigNumberConfigImpl>) kryo.readObject(in, HashMap.class),
+                    in.readBoolean(),
+                    in.readBoolean(),
+                    in.readBoolean(),
+                    in.readString(),
+                    kryo.readObject(in, SchemaFeaturesListWrapper.class).toList(),
+                    kryo.readObject(in, HashMap.class));
+        }
+
+        // Wrapper only used internally for Kryo serialization/deserialization
+        private static class SchemaFeaturesListWrapper
+        {
+            public List<String> requestedFeatureNames;  // CHECKSTYLE IGNORE: Public mutable field
+
+            public List<SchemaFeature> toList()
+            {
+                return requestedFeatureNames.stream()
+                                            .map(name -> SchemaFeatureSet.valueOf(name.toUpperCase()))
+                                            .collect(Collectors.toList());
+            }
+        }
+    }
+
+    protected Set<? extends SidecarInstance> initializeClusterConfig(ClientConfig options)
+    {
+        return Arrays.stream(options.sidecarInstances().split(","))
+                     .map(hostname -> new SidecarInstanceImpl(hostname, options.sidecarPort()))
+                     .collect(Collectors.toSet());
+    }
+
+    protected CompletableFuture<NodeSettings> nodeSettingsFuture(Set<? extends SidecarInstance> clusterConfig,
+                                                                 CompletableFuture<RingResponse> ring)
+    {
+        return sidecar.nodeSettings();
+    }
+
+    protected String getEffectiveCassandraVersionForRead(Set<? extends SidecarInstance> clusterConfig,
+                                                         NodeSettings nodeSettings)
+    {
+        return nodeSettings.releaseVersion();
+    }
+
+    protected void dialHome(@NotNull ClientConfig options)
+    {
+        LOGGER.info("Dial home. clientConfig={}", options);
+    }
+
+    protected void clearSnapshot(Set<? extends SidecarInstance> clusterConfig, @NotNull ClientConfig options)
+    {
+        LOGGER.info("Clearing snapshot at end of Spark job snapshotName={} keyspace={} table={} dc={}",
+                    snapshotName, keyspace, table, datacenter);
+        CountDownLatch latch = new CountDownLatch(clusterConfig.size());
+        try
+        {
+            for (SidecarInstance instance : clusterConfig)
+            {
+                sidecar.clearSnapshot(instance, keyspace, table, snapshotName).whenComplete((resp, throwable) -> {
+                    try
+                    {
+                        if (throwable != null)
+                        {
+                            LOGGER.warn("Failed to clear snapshot on instance hostname={} port={} snapshotName={} keyspace={} table={} datacenter={}",
+                                        instance.hostname(), instance.port(), snapshotName, keyspace, table, datacenter, throwable);
+                        }
+                    }
+                    finally
+                    {
+                        latch.countDown();
+                    }
+                });
+            }
+            await(latch);
+            LOGGER.info("Snapshot cleared snapshotName={} keyspace={} table={} datacenter={}",
+                        snapshotName, keyspace, table, datacenter);
+        }
+        catch (Throwable throwable)
+        {
+            LOGGER.warn("Unexpected exception clearing snapshot snapshotName={} keyspace={} table={} dc={}",
+                        snapshotName, keyspace, table, datacenter, throwable);
+        }
+    }
+
+    /**
+     * Returns the {@link Sizing} object based on the {@code sizing} option provided by the user,
+     * or {@link DefaultSizing} as the default sizing
+     *
+     * @param clusterConfig     the cluster configuration
+     * @param replicationFactor the replication factor
+     * @param options           the {@link ClientConfig} options
+     * @return the {@link Sizing} object based on the {@code sizing} option provided by the user
+     */
+    protected Sizing getSizing(Set<? extends SidecarInstance> clusterConfig,
+                               ReplicationFactor replicationFactor,
+                               ClientConfig options)
+    {
+        return new DefaultSizing(options.numCores());
+    }
+
+    protected void await(CountDownLatch latch)
+    {
+        try
+        {
+            latch.await();
+        }
+        catch (InterruptedException exception)
+        {
+            Thread.currentThread().interrupt();
+            throw new RuntimeException(exception);
+        }
+    }
+
+    public static final class ClientConfig
+    {
+        public static final String SIDECAR_INSTANCES = "sidecar_instances";
+        public static final String KEYSPACE_KEY = "keyspace";
+        public static final String TABLE_KEY = "table";
+        public static final String SNAPSHOT_NAME_KEY = "snapshotName";
+        public static final String DC_KEY = "dc";
+        public static final String CREATE_SNAPSHOT_KEY = "createSnapshot";
+        public static final String CLEAR_SNAPSHOT_KEY = "clearSnapshot";
+        public static final String DEFAULT_PARALLELISM_KEY = "defaultParallelism";
+        public static final String NUM_CORES_KEY = "numCores";
+        public static final String CONSISTENCY_LEVEL_KEY = "consistencyLevel";
+        public static final String ENABLE_STATS_KEY = "enableStats";
+        public static final String LAST_MODIFIED_COLUMN_NAME_KEY = "lastModifiedColumnName";
+        public static final String READ_INDEX_OFFSET_KEY = "readIndexOffset";
+        public static final String SIZING_KEY = "sizing";
+        public static final String SIZING_DEFAULT = "default";
+        public static final String MAX_PARTITION_SIZE_KEY = "maxPartitionSize";
+        public static final String USE_INCREMENTAL_REPAIR = "useIncrementalRepair";
+        public static final String ENABLE_EXPANSION_SHRINK_CHECK_KEY = "enableExpansionShrinkCheck";
+        public static final String SIDECAR_PORT = "sidecar_port";
+        public static final int DEFAULT_SIDECAR_PORT = 9043;
+
+        private final String sidecarInstances;
+        @Nullable
+        private final String keyspace;
+        @Nullable
+        private final String table;
+        private final String snapshotName;
+        private final String datacenter;
+        private final boolean createSnapshot;
+        private final boolean clearSnapshot;
+        private final int defaultParallelism;
+        private final int numCores;
+        private final ConsistencyLevel consistencyLevel;
+        private final Map<String, BigNumberConfigImpl> bigNumberConfigMap;
+        private final boolean enableStats;
+        private final boolean readIndexOffset;
+        private final String sizing;
+        private final int maxPartitionSize;
+        private final boolean useIncrementalRepair;
+        private final List<SchemaFeature> requestedFeatures;
+        private final String lastModifiedTimestampField;
+        private final Boolean enableExpansionShrinkCheck;
+        private final int sidecarPort;
+
+        private ClientConfig(Map<String, String> options)
+        {
+            this.sidecarInstances = MapUtils.getOrThrow(options, SIDECAR_INSTANCES, "sidecar_instances");
+            this.keyspace = MapUtils.getOrThrow(options, KEYSPACE_KEY, "keyspace");
+            this.table = MapUtils.getOrThrow(options, TABLE_KEY, "table");
+            this.snapshotName = MapUtils.getOrDefault(options, SNAPSHOT_NAME_KEY, "sbr_" + UUID.randomUUID().toString().replace("-", ""));
+            this.datacenter = options.get(MapUtils.lowerCaseKey(DC_KEY));
+            this.createSnapshot = MapUtils.getBoolean(options, CREATE_SNAPSHOT_KEY, true);
+            this.clearSnapshot = MapUtils.getBoolean(options, CLEAR_SNAPSHOT_KEY, createSnapshot);
+            this.defaultParallelism = MapUtils.getInt(options, DEFAULT_PARALLELISM_KEY, 1);
+            this.numCores = MapUtils.getInt(options, NUM_CORES_KEY, 1);
+            this.consistencyLevel = Optional.ofNullable(options.get(MapUtils.lowerCaseKey(CONSISTENCY_LEVEL_KEY)))
+                                            .map(ConsistencyLevel::valueOf)
+                                            .orElse(null);
+            this.bigNumberConfigMap = BigNumberConfigImpl.build(options);
+            this.enableStats = MapUtils.getBoolean(options, ENABLE_STATS_KEY, true);
+            this.readIndexOffset = MapUtils.getBoolean(options, READ_INDEX_OFFSET_KEY, true);
+            this.sizing = MapUtils.getOrDefault(options, SIZING_KEY, SIZING_DEFAULT);
+            this.maxPartitionSize = MapUtils.getInt(options, MAX_PARTITION_SIZE_KEY, 1);
+            this.useIncrementalRepair = MapUtils.getBoolean(options, USE_INCREMENTAL_REPAIR, true);
+            this.lastModifiedTimestampField = MapUtils.getOrDefault(options, LAST_MODIFIED_COLUMN_NAME_KEY, null);
+            this.enableExpansionShrinkCheck = MapUtils.getBoolean(options, ENABLE_EXPANSION_SHRINK_CHECK_KEY, false);
+            this.requestedFeatures = initRequestedFeatures(options);
+            this.sidecarPort = MapUtils.getInt(options, SIDECAR_PORT, DEFAULT_SIDECAR_PORT);
+        }
+
+        public String sidecarInstances()
+        {
+            return sidecarInstances;
+        }
+
+        @Nullable
+        public String keyspace()
+        {
+            return keyspace;
+        }
+
+        @Nullable
+        public String table()
+        {
+            return table;
+        }
+
+        public String snapshotName()
+        {
+            return snapshotName;
+        }
+
+        public String datacenter()
+        {
+            return datacenter;
+        }
+
+        public boolean createSnapshot()
+        {
+            return createSnapshot;
+        }
+
+        public boolean clearSnapshot()
+        {
+            return clearSnapshot;
+        }
+
+        public int getDefaultParallelism()
+        {
+            return defaultParallelism;
+        }
+
+        public int numCores()
+        {
+            return numCores;
+        }
+
+        public ConsistencyLevel consistencyLevel()
+        {
+            return consistencyLevel;
+        }
+
+        public Map<String, BigNumberConfigImpl> bigNumberConfigMap()
+        {
+            return bigNumberConfigMap;
+        }
+
+        public boolean enableStats()
+        {
+            return enableStats;
+        }
+
+        public boolean readIndexOffset()
+        {
+            return readIndexOffset;
+        }
+
+        public String sizing()
+        {
+            return sizing;
+        }
+
+        public int maxPartitionSize()
+        {
+            return maxPartitionSize;
+        }
+
+        public boolean useIncrementalRepair()
+        {
+            return useIncrementalRepair;
+        }
+
+        public List<SchemaFeature> requestedFeatures()
+        {
+            return requestedFeatures;
+        }
+
+        public String lastModifiedTimestampField()
+        {
+            return lastModifiedTimestampField;
+        }
+
+        public Boolean enableExpansionShrinkCheck()
+        {
+            return enableExpansionShrinkCheck;
+        }
+
+        public int sidecarPort()
+        {
+            return sidecarPort;
+        }
+
+        public static ClientConfig create(Map<String, String> options)
+        {
+            return new ClientConfig(options);
+        }
+
+        private List<SchemaFeature> initRequestedFeatures(Map<String, String> options)
+        {
+            Map<String, String> optionsCopy = new HashMap<>(options);
+            String lastModifiedColumnName = MapUtils.getOrDefault(options, LAST_MODIFIED_COLUMN_NAME_KEY, null);
+            if (lastModifiedColumnName != null)
+            {
+                optionsCopy.put(SchemaFeatureSet.LAST_MODIFIED_TIMESTAMP.optionName(), "true");
+            }
+            List<SchemaFeature> requestedFeatures = SchemaFeatureSet.initializeFromOptions(optionsCopy);
+            if (lastModifiedColumnName != null)
+            {
+                // Create alias to LAST_MODIFICATION_TIMESTAMP
+                aliasLastModifiedTimestamp(requestedFeatures, lastModifiedColumnName);
+            }
+            return requestedFeatures;
+        }
+    }
+
+    private static void aliasLastModifiedTimestamp(List<SchemaFeature> requestedFeatures, String alias)
+    {
+        SchemaFeature featureAlias = new SchemaFeature()
+        {
+            @Override
+            public String optionName()
+            {
+                return SchemaFeatureSet.LAST_MODIFIED_TIMESTAMP.optionName();
+            }
+
+            @Override
+            public String fieldName()
+            {
+                return alias;
+            }
+
+            @Override
+            public DataType fieldDataType()
+            {
+                return SchemaFeatureSet.LAST_MODIFIED_TIMESTAMP.fieldDataType();
+            }
+
+            @Override
+            public RowBuilder decorate(RowBuilder builder)
+            {
+                return new LastModifiedTimestampDecorator(builder, alias);
+            }
+
+            @Override
+            public boolean fieldNullable()
+            {
+                return SchemaFeatureSet.LAST_MODIFIED_TIMESTAMP.fieldNullable();
+            }
+        };
+        int index = requestedFeatures.indexOf(SchemaFeatureSet.LAST_MODIFIED_TIMESTAMP);
+        if (index >= 0)
+        {
+            requestedFeatures.set(index, featureAlias);
+        }
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/CassandraDataSourceHelper.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/CassandraDataSourceHelper.java
new file mode 100644
index 0000000..5617c05
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/CassandraDataSourceHelper.java
@@ -0,0 +1,131 @@
+/*
+ * 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.cassandra.spark.data;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiConsumer;
+
+import com.google.common.base.Ticker;
+import com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.RemovalListener;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.clients.Sidecar;
+import org.apache.cassandra.clients.SslConfig;
+import org.apache.cassandra.spark.utils.MapUtils;
+import org.apache.spark.SparkConf;
+import org.apache.spark.SparkContext;
+
+/**
+ * A helper class for the CassandraBulkDataSource
+ */
+public final class CassandraDataSourceHelper
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(CassandraDataSourceHelper.class);
+    public static final String CACHE_DATA_LAYER_KEY = "cacheDataLayer";
+    private static Cache<Map<String, String>, CassandraDataLayer> cassandraDataLayerCache;
+
+    public static final int CACHE_HOURS = 12;
+
+    private CassandraDataSourceHelper()
+    {
+        throw new IllegalStateException(getClass() + " is static utility class and shall not be instantiated");
+    }
+
+    public static DataLayer getDataLayer(
+            Map<String, String> options,
+            BiConsumer<CassandraDataLayer, CassandraDataLayer.ClientConfig> initializeDataLayerFn)
+    {
+        CassandraDataLayer.ClientConfig config = CassandraDataLayer.ClientConfig.create(options);
+
+        if (MapUtils.getBoolean(options, CACHE_DATA_LAYER_KEY, false))
+        {
+            // If cacheDataLayer=true, only generate a new CassandraDataLayer.
+            // If any of the DataSourceOptions have changed otherwise use previously cached value.
+            Map<String, String> key = new HashMap<>(options);
+            // Exclude createSnapshot as user may change createSnapshot=false for a snapshotName already created
+            key.remove(CassandraDataLayer.ClientConfig.CREATE_SNAPSHOT_KEY);
+
+            Cache<Map<String, String>, CassandraDataLayer> cassandraDataLayerCache = getCassandraDataLayerCache();
+            CassandraDataLayer cached;
+            try
+            {
+                cached = cassandraDataLayerCache.get(key, () ->
+                        // First thread wins
+                        createAndInitCassandraDataLayer(config, options, initializeDataLayerFn, SparkContext.getOrCreate().getConf()));
+            }
+            catch (ExecutionException exception)
+            {
+                throw new RuntimeException("Unable to initialize CassandraDataLayer", exception);
+            }
+
+            // Return cached value
+            return cached;
+        }
+        else
+        {
+            return createAndInitCassandraDataLayer(config, options, initializeDataLayerFn, SparkContext.getOrCreate().getConf());
+        }
+    }
+
+    protected static Cache<Map<String, String>, CassandraDataLayer> getCassandraDataLayerCache()
+    {
+        if (cassandraDataLayerCache == null)
+        {
+            initCassandraDataSourceCache(Ticker.systemTicker());
+        }
+        return cassandraDataLayerCache;
+    }
+
+    /**
+     * Builds and returns the {@link Cache} for the {@link CassandraDataLayer} instances
+     * with the provided {@code ticker}
+     *
+     * @param ticker the ticker to use for the cache
+     */
+    protected static void initCassandraDataSourceCache(Ticker ticker)
+    {
+        cassandraDataLayerCache = CacheBuilder
+                .newBuilder()
+                .ticker(ticker)
+                .expireAfterWrite(CACHE_HOURS, TimeUnit.HOURS)
+                .removalListener((RemovalListener<Map<String, String>, CassandraDataLayer>) notification ->
+                        LOGGER.debug("Removed entry '{}' from CassandraDataSourceCache", notification.getValue()))
+                .build();
+    }
+
+    protected static CassandraDataLayer createAndInitCassandraDataLayer(
+            CassandraDataLayer.ClientConfig config,
+            Map<String, String> options,
+            BiConsumer<CassandraDataLayer, CassandraDataLayer.ClientConfig> initializeDataLayerFn,
+            SparkConf conf)
+    {
+        CassandraDataLayer dataLayer = new CassandraDataLayer(config,
+                                                              Sidecar.ClientConfig.create(options),
+                                                              SslConfig.create(options));
+        initializeDataLayerFn.accept(dataLayer, config);
+        return dataLayer;
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/DefaultSizing.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/DefaultSizing.java
new file mode 100644
index 0000000..82bee56
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/DefaultSizing.java
@@ -0,0 +1,47 @@
+/*
+ * 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.cassandra.spark.data;
+
+/**
+ * Default sizing based on the number of cores option
+ */
+public class DefaultSizing implements Sizing
+{
+    private final int numCores;
+
+    /**
+     * Constructs a new object with the given {@code numCores}
+     *
+     * @param numCores the number of cores for the execution
+     */
+    public DefaultSizing(int numCores)
+    {
+        this.numCores = numCores;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public int getEffectiveNumberOfCores()
+    {
+        return numCores;
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/FileSystemSSTable.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/FileSystemSSTable.java
new file mode 100644
index 0000000..512f4c9
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/FileSystemSSTable.java
@@ -0,0 +1,102 @@
+/*
+ * 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.cassandra.spark.data;
+
+import java.io.BufferedInputStream;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Path;
+import java.util.function.Supplier;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.spark.stats.Stats;
+import org.apache.cassandra.spark.utils.ThrowableUtils;
+import org.apache.cassandra.spark.utils.streaming.SSTableInputStream;
+
+class FileSystemSSTable extends SSTable
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(FileSystemSSTable.class);
+
+    private final Path dataFilePath;
+    private final boolean useSSTableInputStream;
+    private final Supplier<Stats> stats;
+
+    FileSystemSSTable(Path dataFilePath, boolean useSSTableInputStream, Supplier<Stats> stats)
+    {
+        this.dataFilePath = dataFilePath;
+        this.useSSTableInputStream = useSSTableInputStream;
+        this.stats = stats;
+    }
+
+    @Override
+    protected InputStream openInputStream(FileType fileType)
+    {
+        Path filePath = FileType.resolveComponentFile(fileType, dataFilePath);
+        if (filePath == null)
+        {
+            return null;
+        }
+        try
+        {
+            return useSSTableInputStream
+                    ? new SSTableInputStream<>(new FileSystemSource(this, fileType, filePath), stats.get())
+                    : new BufferedInputStream(new FileInputStream(filePath.toFile()));
+        }
+        catch (FileNotFoundException exception)
+        {
+            return null;
+        }
+        catch (IOException exception)
+        {
+            Throwable cause = ThrowableUtils.rootCause(exception);
+            LOGGER.warn("IOException reading local sstable", cause);
+            throw new RuntimeException(cause);
+        }
+    }
+
+    @Override
+    public boolean isMissing(FileType fileType)
+    {
+        return FileType.resolveComponentFile(fileType, dataFilePath) == null;
+    }
+
+    @Override
+    public String getDataFileName()
+    {
+        return dataFilePath.getFileName().toString();
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return dataFilePath.hashCode();
+    }
+
+    @Override
+    public boolean equals(Object other)
+    {
+        return other instanceof FileSystemSSTable
+            && this.dataFilePath.equals(((FileSystemSSTable) other).dataFilePath);
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/FileSystemSource.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/FileSystemSource.java
new file mode 100644
index 0000000..0efc1ed
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/FileSystemSource.java
@@ -0,0 +1,157 @@
+/*
+ * 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.cassandra.spark.data;
+
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.spark.utils.IOUtils;
+import org.apache.cassandra.spark.utils.streaming.SSTableSource;
+import org.apache.cassandra.spark.utils.streaming.StreamBuffer;
+import org.apache.cassandra.spark.utils.streaming.StreamConsumer;
+
+class FileSystemSource implements SSTableSource<FileSystemSSTable>, AutoCloseable
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(FileSystemSource.class);
+    static final ExecutorService FILE_IO_EXECUTOR =
+            Executors.newFixedThreadPool(4, new ThreadFactoryBuilder().setNameFormat("file-io-%d")
+                                                                      .setDaemon(true)
+                                                                      .build());
+    protected static final ExecutorService COMMIT_LOG_EXECUTOR =
+            Executors.newFixedThreadPool(4, new ThreadFactoryBuilder().setNameFormat("commit-log-%d")
+                                                                      .setDaemon(true)
+                                                                      .build());
+
+    private final FileSystemSSTable ssTable;
+    private final RandomAccessFile file;
+    private final FileType fileType;
+    private final long length;
+
+    FileSystemSource(FileSystemSSTable sstable, FileType fileType, Path path) throws IOException
+    {
+        this.ssTable = sstable;
+        this.fileType = fileType;
+        this.length = Files.size(path);
+        this.file = new RandomAccessFile(path.toFile(), "r");
+    }
+
+    @Override
+    public long maxBufferSize()
+    {
+        return chunkBufferSize() * 4;
+    }
+
+    @Override
+    public long chunkBufferSize()
+    {
+        return 16 * 1024;
+    }
+
+    @Override
+    public long headerChunkSize()
+    {
+        return fileType == FileType.COMMITLOG ? IOUtils.DEFAULT_CDC_BUFFER_SIZE : chunkBufferSize();
+    }
+
+    public ExecutorService executor()
+    {
+        return FILE_IO_EXECUTOR;
+    }
+
+    @Override
+    public void request(long start, long end, StreamConsumer consumer)
+    {
+        executor().submit(() -> {
+            boolean close = length <= end;
+            try
+            {
+                // Start-end range is inclusive but on the final request end == length so we need to exclude
+                int increment = close ? 0 : 1;
+                byte[] bytes = new byte[(int) (end - start + increment)];
+                if (file.getChannel().read(ByteBuffer.wrap(bytes), start) >= 0)
+                {
+                    consumer.onRead(StreamBuffer.wrap(bytes));
+                    consumer.onEnd();
+                }
+                else
+                {
+                    close = true;
+                }
+            }
+            catch (Throwable throwable)
+            {
+                close = true;
+                consumer.onError(throwable);
+            }
+            finally
+            {
+                if (close)
+                {
+                    closeSafe();
+                }
+            }
+        });
+    }
+
+    public FileSystemSSTable sstable()
+    {
+        return ssTable;
+    }
+
+    public FileType fileType()
+    {
+        return fileType;
+    }
+
+    public long size()
+    {
+        return length;
+    }
+
+    private void closeSafe()
+    {
+        try
+        {
+            close();
+        }
+        catch (Exception exception)
+        {
+            LOGGER.warn("Exception closing InputStream", exception);
+        }
+    }
+
+    @Override
+    public void close() throws IOException
+    {
+        if (file != null)
+        {
+            file.close();
+        }
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/LocalCommitLog.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/LocalCommitLog.java
new file mode 100644
index 0000000..8fea060
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/LocalCommitLog.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.cassandra.spark.data;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.concurrent.ExecutorService;
+
+import org.apache.cassandra.spark.cdc.CommitLog;
+import org.apache.cassandra.spark.data.partitioner.CassandraInstance;
+import org.apache.cassandra.spark.utils.streaming.SSTableSource;
+
+public class LocalCommitLog implements CommitLog
+{
+    final long length;
+    final String name;
+    final String path;
+    final FileSystemSource source;
+    final CassandraInstance instance;
+
+    public LocalCommitLog(File file)
+    {
+        this.name = file.getName();
+        this.path = file.getPath();
+        this.length = file.length();
+        this.instance = new CassandraInstance("0", "local-instance", "DC1");
+
+        try
+        {
+            this.source = new FileSystemSource(null, FileType.COMMITLOG, file.toPath())
+            {
+                @Override
+                public ExecutorService executor()
+                {
+                    return COMMIT_LOG_EXECUTOR;
+                }
+
+                @Override
+                public long size()
+                {
+                    return length;
+                }
+            };
+        }
+        catch (IOException exception)
+        {
+            throw new RuntimeException(exception);
+        }
+    }
+
+    @Override
+    public String name()
+    {
+        return name;
+    }
+
+    @Override
+    public String path()
+    {
+        return path;
+    }
+
+    @Override
+    public long maxOffset()
+    {
+        return length;
+    }
+
+    @Override
+    public long length()
+    {
+        return length;
+    }
+
+    @Override
+    public SSTableSource<? extends SSTable> source()
+    {
+        return source;
+    }
+
+    @Override
+    public CassandraInstance instance()
+    {
+        return instance;
+    }
+
+    @Override
+    public void close() throws Exception
+    {
+        source.close();
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/LocalDataLayer.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/LocalDataLayer.java
new file mode 100644
index 0000000..74a8daa
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/LocalDataLayer.java
@@ -0,0 +1,479 @@
+/*
+ * 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.cassandra.spark.data;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.Serializable;
+import java.lang.reflect.Field;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableMap;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.lang.builder.EqualsBuilder;
+import org.apache.commons.lang.builder.HashCodeBuilder;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+import org.apache.cassandra.bridge.CassandraBridge;
+import org.apache.cassandra.bridge.CassandraBridgeFactory;
+import org.apache.cassandra.bridge.CassandraVersion;
+import org.apache.cassandra.spark.cdc.CommitLogProvider;
+import org.apache.cassandra.spark.cdc.TableIdLookup;
+import org.apache.cassandra.spark.cdc.watermarker.InMemoryWatermarker;
+import org.apache.cassandra.spark.cdc.watermarker.Watermarker;
+import org.apache.cassandra.spark.config.SchemaFeature;
+import org.apache.cassandra.spark.config.SchemaFeatureSet;
+import org.apache.cassandra.spark.data.partitioner.Partitioner;
+import org.apache.cassandra.spark.sparksql.filters.PartitionKeyFilter;
+import org.apache.cassandra.spark.sparksql.filters.SparkRangeFilter;
+import org.apache.cassandra.spark.stats.Stats;
+import org.apache.cassandra.spark.utils.Throwing;
+import org.apache.parquet.Strings;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Basic DataLayer implementation to read SSTables from local file system. Mostly used for testing.
+ */
+@SuppressWarnings({"unused", "WeakerAccess"})
+public class LocalDataLayer extends DataLayer implements Serializable
+{
+    public static final long serialVersionUID = 42L;
+
+    private transient CassandraBridge bridge;
+    private Partitioner partitioner;
+    private CqlTable cqlTable;
+    private String jobId;
+    private String statsClass;
+    private transient volatile Stats stats = null;
+    private List<SchemaFeature> requestedFeatures;
+    private boolean useSSTableInputStream;
+    private String[] paths;
+    private int minimumReplicasPerMutation = 1;
+
+    @Nullable
+    private static Stats loadStats(@Nullable String statsClass)
+    {
+        if (Strings.isNullOrEmpty(statsClass))
+        {
+            return null;
+        }
+        // For tests it's useful to inject a custom stats instance to collect & verify metrics
+        try
+        {
+            int index = statsClass.lastIndexOf(".");
+            String className = statsClass.substring(0, index);
+            String fieldName = statsClass.substring(index + 1);
+            Field field = Class.forName(className).getDeclaredField(fieldName);
+            return (Stats) field.get(null);
+        }
+        catch (ClassNotFoundException | NoSuchFieldException | IllegalAccessException exception)
+        {
+            throw new RuntimeException(exception);
+        }
+    }
+
+    /**
+     * Returns the lower-cased key using {@link Locale#ROOT}
+     *
+     * @param key the key
+     * @return the lower-cased key using {@link Locale#ROOT}
+     */
+    @Nullable
+    private static String lowerCaseKey(@Nullable String key)
+    {
+        return key != null ? key.toLowerCase(Locale.ROOT) : null;
+    }
+
+    @NotNull
+    private static String getOrThrow(@NotNull Map<String, String> options, @Nullable String key)
+    {
+        String value = options.get(key);
+        if (value != null)
+        {
+            return value;
+        }
+        else
+        {
+            throw new IllegalArgumentException("Value for key '" + key + "' is missing from the options map");
+        }
+    }
+
+    @SuppressWarnings("SameParameterValue")
+    private static boolean getBoolean(@NotNull Map<String, String> options, @Nullable String key, boolean defaultValue)
+    {
+        String value = options.get(key);
+        // We can't use `Boolean.parseBoolean` here, as it returns false for invalid strings
+        if (value == null)
+        {
+            return defaultValue;
+        }
+        else if (value.equalsIgnoreCase("true"))
+        {
+            return true;
+        }
+        else if (value.equalsIgnoreCase("false"))
+        {
+            return false;
+        }
+        else
+        {
+            throw new IllegalArgumentException("Value '" + value + "' for key '" + key + "' is not a boolean string");
+        }
+    }
+
+    /**
+     * Builds a new {@link DataLayer} from the {@code options} map. The keys for the map
+     * must be lower-cased to guarantee compatibility with maps where the keys are all
+     * lower-cased.
+     *
+     * @param options the map with options
+     * @return a new {@link DataLayer}
+     */
+    public static LocalDataLayer from(Map<String, String> options)
+    {
+        // Keys need to be lower-cased to access the map
+        return new LocalDataLayer(
+                CassandraVersion.valueOf(options.getOrDefault(lowerCaseKey("version"), CassandraVersion.FOURZERO.name())),
+                Partitioner.valueOf(options.getOrDefault(lowerCaseKey("partitioner"), Partitioner.Murmur3Partitioner.name())),
+                getOrThrow(options, lowerCaseKey("keyspace")),
+                getOrThrow(options, lowerCaseKey("createStmt")),
+                Arrays.stream(options.getOrDefault(lowerCaseKey("udts"), "").split("\n"))
+                      .filter(StringUtils::isNotEmpty)
+                      .collect(Collectors.toSet()),
+                SchemaFeatureSet.initializeFromOptions(options),
+                getBoolean(options, lowerCaseKey("useSSTableInputStream"), false),
+                options.get(lowerCaseKey("statsClass")),
+                getOrThrow(options, lowerCaseKey("dirs")).split(","));
+    }
+
+    public LocalDataLayer(@NotNull CassandraVersion version,
+                          @NotNull String keyspace,
+                          @NotNull String createStatement,
+                          String... paths)
+    {
+        this(version,
+             Partitioner.Murmur3Partitioner,
+             keyspace,
+             createStatement,
+             Collections.emptySet(),
+             Collections.emptyList(),
+             false,
+             null,
+             paths);
+    }
+
+    // CHECKSTYLE IGNORE: Constructor with many parameters
+    public LocalDataLayer(@NotNull CassandraVersion version,
+                          @NotNull Partitioner partitioner,
+                          @NotNull String keyspace,
+                          @NotNull String createStatement,
+                          @NotNull Set<String> udts,
+                          @NotNull List<SchemaFeature> requestedFeatures,
+                          boolean useSSTableInputStream,
+                          @Nullable String statsClass,
+                          String... paths)
+    {
+        this.bridge = CassandraBridgeFactory.get(version);
+        this.partitioner = partitioner;
+        this.cqlTable = bridge().buildSchema(createStatement,
+                                             keyspace,
+                                             new ReplicationFactor(ReplicationFactor.ReplicationStrategy.SimpleStrategy,
+                                                                   ImmutableMap.of("replication_factor", 1)),
+                                             partitioner,
+                                             udts);
+        this.jobId = UUID.randomUUID().toString();
+        this.requestedFeatures = requestedFeatures;
+        this.useSSTableInputStream = useSSTableInputStream;
+        this.statsClass = statsClass;
+        this.paths = paths;
+    }
+
+    // For serialization
+    private LocalDataLayer(@NotNull CassandraVersion version,
+                           @NotNull Partitioner partitioner,
+                           @NotNull CqlTable cqlTable,
+                           @NotNull String jobId,
+                           @NotNull List<SchemaFeature> requestedFeatures,
+                           boolean useSSTableInputStream,
+                           @Nullable String statsClass,
+                           String... paths)
+    {
+        this.bridge = CassandraBridgeFactory.get(version);
+        this.partitioner = partitioner;
+        this.cqlTable = cqlTable;
+        this.jobId = jobId;
+        this.requestedFeatures = requestedFeatures;
+        this.useSSTableInputStream = useSSTableInputStream;
+        this.statsClass = statsClass;
+        this.paths = paths;
+    }
+
+    @Override
+    public List<SchemaFeature> requestedFeatures()
+    {
+        return requestedFeatures;
+    }
+
+    @Override
+    public CassandraBridge bridge()
+    {
+        return bridge;
+    }
+
+    @Override
+    public Partitioner partitioner()
+    {
+        return partitioner;
+    }
+
+    @Override
+    public int partitionCount()
+    {
+        return 1;
+    }
+
+    @Override
+    public boolean isInPartition(int partitionId, BigInteger token, ByteBuffer key)
+    {
+        return true;
+    }
+
+    @Override
+    public CqlTable cqlTable()
+    {
+        return cqlTable;
+    }
+
+    @Override
+    public synchronized Stats stats()
+    {
+        if (stats == null)
+        {
+            stats = loadStats(statsClass);
+        }
+        if (stats == null)
+        {
+            stats = super.stats();
+        }
+        return stats;
+    }
+
+    @Override
+    public SSTablesSupplier sstables(int partitionId,
+                                     @Nullable SparkRangeFilter sparkRangeFilter,
+                                     @NotNull List<PartitionKeyFilter> partitionKeyFilters)
+    {
+        return new BasicSupplier(Arrays
+                .stream(paths)
+                .map(Paths::get)
+                .flatMap(Throwing.function(Files::list))
+                .filter(path -> path.getFileName().toString().endsWith("-" + FileType.DATA.getFileSuffix()))
+                .map(path -> new FileSystemSSTable(path, useSSTableInputStream, this::stats))
+                .collect(Collectors.toSet()));
+    }
+
+    @VisibleForTesting
+    public LocalDataLayer withMinimumReplicasPerMutation(int minimumReplicasPerMutation)
+    {
+        this.minimumReplicasPerMutation = minimumReplicasPerMutation;
+        return this;
+    }
+
+    @Override
+    public int minimumReplicasForCdc()
+    {
+        return minimumReplicasPerMutation;
+    }
+
+    public String jobId()
+    {
+        return jobId;
+    }
+
+    @Override
+    public Watermarker cdcWatermarker()
+    {
+        return InMemoryWatermarker.INSTANCE;
+    }
+
+    @Override
+    public CommitLogProvider commitLogs(int partitionId)
+    {
+        return () -> Arrays.stream(paths)
+                           .map(Paths::get)
+                           .flatMap(LocalDataLayer::listPath)
+                           .map(Path::toFile)
+                           .map(LocalCommitLog::new);
+    }
+
+    @Override
+    public TableIdLookup tableIdLookup()
+    {
+        // Do nothing, because in tests the tableId in the CommitLog should match in the Schema instance in JVM
+        return (keyspace, table) -> null;
+    }
+
+    @Override
+    protected ExecutorService executorService()
+    {
+        return FileSystemSource.FILE_IO_EXECUTOR;
+    }
+
+    private static Stream<Path> listPath(Path path)
+    {
+        try
+        {
+            return Files.list(path);
+        }
+        catch (IOException exception)
+        {
+            throw new RuntimeException(exception);
+        }
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return new HashCodeBuilder()
+               .append(cqlTable)
+               .append(paths)
+               .append(version())
+               .toHashCode();
+    }
+
+    @Override
+    public boolean equals(Object other)
+    {
+        if (other == null)
+        {
+            return false;
+        }
+        if (this == other)
+        {
+            return true;
+        }
+        if (this.getClass() != other.getClass())
+        {
+            return false;
+        }
+
+        LocalDataLayer that = (LocalDataLayer) other;
+        return new EqualsBuilder()
+               .append(this.cqlTable, that.cqlTable)
+               .append(this.paths, that.paths)
+               .append(this.version(), that.version())
+               .isEquals();
+    }
+
+    // JDK Serialization
+
+    private void writeObject(ObjectOutputStream out) throws IOException, ClassNotFoundException
+    {
+        // Falling back to JDK serialization
+        out.writeUTF(this.version().name());
+        out.writeObject(this.partitioner);
+        bridge.javaSerialize(out, this.cqlTable);  // Delegate (de-)serialization of version-specific objects to the Cassandra Bridge
+        out.writeUTF(this.jobId);
+        out.writeObject(this.requestedFeatures.stream()
+                                              .map(SchemaFeature::toString)
+                                              .toArray(String[]::new));
+        out.writeBoolean(this.useSSTableInputStream);
+        out.writeBoolean(this.statsClass != null);
+        if (this.statsClass != null)
+        {
+            out.writeUTF(this.statsClass);
+        }
+        out.writeObject(this.paths);
+        out.writeInt(this.minimumReplicasPerMutation);
+    }
+
+    private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException
+    {
+        // Falling back to JDK deserialization
+        this.bridge = CassandraBridgeFactory.get(CassandraVersion.valueOf(in.readUTF()));
+        this.partitioner = (Partitioner) in.readObject();
+        this.cqlTable = bridge.javaDeserialize(in, CqlTable.class);  // Delegate (de-)serialization of version-specific objects to the Cassandra Bridge
+        this.jobId = in.readUTF();
+        this.requestedFeatures = Arrays.stream((String[]) in.readObject())
+                                       .map(SchemaFeatureSet::valueOf)
+                                       .collect(Collectors.toList());
+        this.useSSTableInputStream = in.readBoolean();
+        this.statsClass = in.readBoolean() ? in.readUTF() : null;
+        this.paths = (String[]) in.readObject();
+        this.minimumReplicasPerMutation = in.readInt();
+    }
+
+    // Kryo Serialization
+
+    public static class Serializer extends com.esotericsoftware.kryo.Serializer<LocalDataLayer>
+    {
+        @Override
+        public void write(Kryo kryo, Output out, LocalDataLayer object)
+        {
+            kryo.writeObject(out, object.version());
+            kryo.writeObject(out, object.partitioner);
+            kryo.writeObject(out, object.cqlTable);
+            out.writeString(object.jobId);
+            kryo.writeObject(out, object.requestedFeatures.stream()
+                                                          .map(SchemaFeature::toString)
+                                                          .toArray(String[]::new));
+            out.writeBoolean(object.useSSTableInputStream);
+            out.writeString(object.statsClass);
+            kryo.writeObject(out, object.paths);
+            out.writeInt(object.minimumReplicasPerMutation);
+        }
+
+        @Override
+        public LocalDataLayer read(Kryo kryo, Input in, Class<LocalDataLayer> type)
+        {
+            return new LocalDataLayer(
+                    kryo.readObject(in, CassandraVersion.class),
+                    kryo.readObject(in, Partitioner.class),
+                    kryo.readObject(in, CqlTable.class),
+                    in.readString(),
+                    Arrays.stream(kryo.readObject(in, String[].class))
+                          .map(SchemaFeatureSet::valueOf)
+                          .collect(Collectors.toList()),
+                    in.readBoolean(),
+                    in.readString(),
+                    kryo.readObject(in, String[].class)
+            ).withMinimumReplicasPerMutation(in.readInt());
+        }
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/PartitionedDataLayer.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/PartitionedDataLayer.java
new file mode 100644
index 0000000..8cfc33a
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/PartitionedDataLayer.java
@@ -0,0 +1,580 @@
+/*
+ * 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.cassandra.spark.data;
+
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Range;
+import org.apache.commons.lang.builder.EqualsBuilder;
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.bridge.TokenRange;
+import org.apache.cassandra.spark.cdc.CommitLog;
+import org.apache.cassandra.spark.cdc.CommitLogProvider;
+import org.apache.cassandra.spark.data.partitioner.CassandraInstance;
+import org.apache.cassandra.spark.data.partitioner.CassandraRing;
+import org.apache.cassandra.spark.data.partitioner.ConsistencyLevel;
+import org.apache.cassandra.spark.data.partitioner.MultipleReplicas;
+import org.apache.cassandra.spark.data.partitioner.NotEnoughReplicasException;
+import org.apache.cassandra.spark.data.partitioner.Partitioner;
+import org.apache.cassandra.spark.data.partitioner.SingleReplica;
+import org.apache.cassandra.spark.data.partitioner.TokenPartitioner;
+import org.apache.cassandra.spark.sparksql.NoMatchFoundException;
+import org.apache.cassandra.spark.sparksql.filters.PartitionKeyFilter;
+import org.apache.cassandra.spark.sparksql.filters.SparkRangeFilter;
+import org.apache.cassandra.spark.stats.Stats;
+import org.apache.cassandra.spark.utils.FutureUtils;
+import org.apache.cassandra.spark.utils.RangeUtils;
+import org.apache.cassandra.spark.utils.ThrowableUtils;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * DataLayer that partitions token range by the number of Spark partitions
+ * and only lists SSTables overlapping with range
+ */
+@SuppressWarnings("WeakerAccess")
+public abstract class PartitionedDataLayer extends DataLayer
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(PartitionedDataLayer.class);
+    private static final ConsistencyLevel DEFAULT_CONSISTENCY_LEVEL = ConsistencyLevel.LOCAL_QUORUM;
+
+    @NotNull
+    protected ConsistencyLevel consistencyLevel;
+    protected String datacenter;
+
+    public enum AvailabilityHint
+    {
+        // 0 means high priority
+        UP(0), MOVING(1), LEAVING(1), UNKNOWN(2), JOINING(2), DOWN(2);
+
+        private final int priority;
+
+        AvailabilityHint(int priority)
+        {
+            this.priority = priority;
+        }
+
+        public static final Comparator<AvailabilityHint> AVAILABILITY_HINT_COMPARATOR =
+                Comparator.comparingInt((AvailabilityHint other) -> other.priority).reversed();
+
+        public static AvailabilityHint fromState(String status, String state)
+        {
+            if (status.equalsIgnoreCase(AvailabilityHint.DOWN.name()))
+            {
+                return AvailabilityHint.DOWN;
+            }
+
+            if (status.equalsIgnoreCase(AvailabilityHint.UNKNOWN.name()))
+            {
+                return AvailabilityHint.UNKNOWN;
+            }
+
+            if (state.equalsIgnoreCase("NORMAL"))
+            {
+                return AvailabilityHint.valueOf(status.toUpperCase());
+            }
+            if (state.equalsIgnoreCase(AvailabilityHint.MOVING.name()))
+            {
+                return AvailabilityHint.MOVING;
+            }
+            if (state.equalsIgnoreCase(AvailabilityHint.LEAVING.name()))
+            {
+                return AvailabilityHint.LEAVING;
+            }
+            if (state.equalsIgnoreCase("STARTING"))
+            {
+                return AvailabilityHint.valueOf(status.toUpperCase());
+            }
+            if (state.equalsIgnoreCase(AvailabilityHint.JOINING.name()))
+            {
+                return AvailabilityHint.JOINING;
+            }
+
+            return AvailabilityHint.UNKNOWN;
+        }
+    }
+
+    public PartitionedDataLayer(@Nullable ConsistencyLevel consistencyLevel, @Nullable String datacenter)
+    {
+        this.consistencyLevel = consistencyLevel != null ? consistencyLevel : DEFAULT_CONSISTENCY_LEVEL;
+        this.datacenter = datacenter;
+
+        if (consistencyLevel == ConsistencyLevel.SERIAL || consistencyLevel == ConsistencyLevel.LOCAL_SERIAL)
+        {
+            throw new IllegalArgumentException("SERIAL or LOCAL_SERIAL are invalid consistency levels for the Bulk Reader");
+        }
+        if (consistencyLevel == ConsistencyLevel.EACH_QUORUM)
+        {
+            throw new UnsupportedOperationException("EACH_QUORUM has not been implemented yet");
+        }
+    }
+
+    protected void validateReplicationFactor(@NotNull ReplicationFactor replicationFactor)
+    {
+        validateReplicationFactor(consistencyLevel, replicationFactor, datacenter);
+    }
+
+    @VisibleForTesting
+    public static void validateReplicationFactor(@NotNull ConsistencyLevel consistencyLevel,
+                                                 @NotNull ReplicationFactor replicationFactor,
+                                                 @Nullable String dc)
+    {
+        if (replicationFactor.getReplicationStrategy() != ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy)
+        {
+            return;
+        }
+        // Single DC and no DC specified so use only DC in replication factor
+        if (dc == null && replicationFactor.getOptions().size() == 1)
+        {
+            return;
+        }
+        Preconditions.checkArgument(dc != null || !consistencyLevel.isDCLocal,
+                                    "A DC must be specified for DC local consistency level " + consistencyLevel.name());
+        if (dc == null)
+        {
+            return;
+        }
+        Preconditions.checkArgument(replicationFactor.getOptions().containsKey(dc),
+                                    "DC %s not found in replication factor %s",
+                                    dc, replicationFactor.getOptions().keySet());
+        Preconditions.checkArgument(replicationFactor.getOptions().get(dc) > 0,
+                                    "Cannot read from DC %s with non-positive replication factor %d",
+                                    dc, replicationFactor.getOptions().get(dc));
+    }
+
+    public abstract CompletableFuture<Stream<SSTable>> listInstance(int partitionId, @NotNull Range<BigInteger> range, @NotNull CassandraInstance instance);
+
+    public abstract CassandraRing ring();
+
+    public abstract TokenPartitioner tokenPartitioner();
+
+    @Override
+    public int partitionCount()
+    {
+        return tokenPartitioner().numPartitions();
+    }
+
+    @Override
+    public Partitioner partitioner()
+    {
+        return ring().partitioner();
+    }
+
+    @Override
+    public boolean isInPartition(int partitionId, BigInteger token, ByteBuffer key)
+    {
+        return tokenPartitioner().isInPartition(token, key, partitionId);
+    }
+
+    @Override
+    public SparkRangeFilter sparkRangeFilter(int partitionId)
+    {
+        Map<Integer, Range<BigInteger>> reversePartitionMap = tokenPartitioner().reversePartitionMap();
+        Range<BigInteger> sparkTokenRange = reversePartitionMap.get(partitionId);
+        if (sparkTokenRange == null)
+        {
+            LOGGER.error("Unable to find the sparkTokenRange for partitionId={} in reversePartitionMap={}",
+                         partitionId, reversePartitionMap);
+            throw new IllegalStateException(
+                    String.format("Unable to find sparkTokenRange for partitionId=%d in the reverse partition map",
+                                  partitionId));
+        }
+        return SparkRangeFilter.create(RangeUtils.toTokenRange(sparkTokenRange));
+    }
+
+    @Override
+    public List<PartitionKeyFilter> partitionKeyFiltersInRange(
+            int partitionId,
+            List<PartitionKeyFilter> filters) throws NoMatchFoundException
+    {
+        // We only need to worry about Partition key filters that overlap with this Spark workers token range
+        SparkRangeFilter rangeFilter = sparkRangeFilter(partitionId);
+        TokenRange sparkTokenRange = rangeFilter.tokenRange();
+
+        List<PartitionKeyFilter> filtersInRange = filters.stream()
+                                                         .filter(filter -> filter.overlaps(sparkTokenRange))
+                                                         .collect(Collectors.toList());
+
+        if (!filters.isEmpty() && filtersInRange.isEmpty())
+        {
+            LOGGER.info("None of the partition key filters overlap with Spark partition token range firstToken={} lastToken{}",
+                        sparkTokenRange.lowerEndpoint(), sparkTokenRange.upperEndpoint());
+            throw new NoMatchFoundException();
+        }
+        return filterNonIntersectingSSTables() ? filtersInRange : filters;
+    }
+
+    public ConsistencyLevel consistencylevel()
+    {
+        return consistencyLevel;
+    }
+
+    @Override
+    public SSTablesSupplier sstables(int partitionId,
+                                     @Nullable SparkRangeFilter sparkRangeFilter,
+                                     @NotNull List<PartitionKeyFilter> partitionKeyFilters)
+    {
+        // Get token range for Spark partition
+        TokenPartitioner tokenPartitioner = tokenPartitioner();
+        if (partitionId < 0 || partitionId >= tokenPartitioner.numPartitions())
+        {
+            throw new IllegalStateException("PartitionId outside expected range: " + partitionId);
+        }
+
+        // Get all replicas overlapping partition token range
+        Range<BigInteger> range = tokenPartitioner.getTokenRange(partitionId);
+        CassandraRing ring = ring();
+        ReplicationFactor replicationFactor = ring.replicationFactor();
+        validateReplicationFactor(replicationFactor);
+        Map<Range<BigInteger>, List<CassandraInstance>> instRanges;
+        Map<Range<BigInteger>, List<CassandraInstance>> subRanges = ring().getSubRanges(range).asMapOfRanges();
+        if (partitionKeyFilters.isEmpty())
+        {
+            instRanges = subRanges;
+        }
+        else
+        {
+            instRanges = new HashMap<>();
+            subRanges.keySet().forEach(instRange -> {
+                TokenRange tokenRange = RangeUtils.toTokenRange(instRange);
+                if (partitionKeyFilters.stream().anyMatch(filter -> filter.overlaps(tokenRange)))
+                {
+                    instRanges.putIfAbsent(instRange, subRanges.get(instRange));
+                }
+            });
+        }
+
+        Set<CassandraInstance> replicas = PartitionedDataLayer.rangesToReplicas(consistencyLevel, datacenter, instRanges);
+        LOGGER.info("Creating partitioned SSTablesSupplier for Spark partition partitionId={} rangeLower={} rangeUpper={} numReplicas={}",
+                    partitionId, range.lowerEndpoint(), range.upperEndpoint(), replicas.size());
+
+        // Use consistency level and replication factor to calculate min number of replicas required
+        // to satisfy consistency level; split replicas into 'primary' and 'backup' replicas,
+        // attempt on primary replicas and use backups to retry in the event of a failure
+        int minReplicas = consistencyLevel.blockFor(replicationFactor, datacenter);
+        ReplicaSet replicaSet = PartitionedDataLayer.splitReplicas(
+                consistencyLevel, datacenter, instRanges, replicas, this::getAvailability, minReplicas, partitionId);
+        if (replicaSet.primary().size() < minReplicas)
+        {
+            // Could not find enough primary replicas to meet consistency level
+            assert replicaSet.backup().isEmpty();
+            throw new NotEnoughReplicasException(consistencyLevel, range, minReplicas, replicas.size(), datacenter);
+        }
+
+        ExecutorService executor = executorService();
+        Stats stats = stats();
+        Set<SingleReplica> primaryReplicas = replicaSet.primary().stream()
+                .map(instance -> new SingleReplica(instance, this, range, partitionId, executor, stats, replicaSet.isRepairPrimary(instance)))
+                .collect(Collectors.toSet());
+        Set<SingleReplica> backupReplicas = replicaSet.backup().stream()
+                .map(instance -> new SingleReplica(instance, this, range, partitionId, executor, stats, true))
+                .collect(Collectors.toSet());
+
+        return new MultipleReplicas(primaryReplicas, backupReplicas, stats);
+    }
+
+    /**
+     * Overridable method setting whether the PartitionedDataLayer should filter out SSTables
+     * that do not intersect with the Spark partition token range
+     *
+     * @return true if we should filter
+     */
+    public boolean filterNonIntersectingSSTables()
+    {
+        return true;
+    }
+
+    /**
+     * Data Layer can override this method to hint availability of a Cassandra instance so Bulk Reader attempts
+     * UP instances first, and avoids instances known to be down e.g. if create snapshot request already failed
+     *
+     * @param instance a cassandra instance
+     * @return availability hint
+     */
+    protected AvailabilityHint getAvailability(CassandraInstance instance)
+    {
+        return AvailabilityHint.UNKNOWN;
+    }
+
+    static Set<CassandraInstance> rangesToReplicas(@NotNull ConsistencyLevel consistencyLevel,
+                                                   @Nullable String dataCenter,
+                                                   @NotNull Map<Range<BigInteger>, List<CassandraInstance>> ranges)
+    {
+        return ranges.values().stream()
+                .flatMap(Collection::stream)
+                .filter(instance -> !consistencyLevel.isDCLocal || dataCenter == null || instance.dataCenter().equals(dataCenter))
+                .collect(Collectors.toSet());
+    }
+
+    /**
+     * Split the replicas overlapping with the Spark worker's token range based on availability hint so that we
+     * achieve consistency
+     *
+     * @param consistencyLevel user set consistency level
+     * @param dataCenter       data center to read from
+     * @param ranges           all the token ranges owned by this Spark worker, and associated replicas
+     * @param replicas         all the replicas we can read from
+     * @param availability     availability hint provider for each CassandraInstance
+     * @param minReplicas      minimum number of replicas to achieve consistency
+     * @param partitionId      Spark worker partitionId
+     * @return a set of primary and backup replicas to read from
+     * @throws NotEnoughReplicasException thrown when insufficient primary replicas selected to achieve
+     *                                    consistency level for any sub-range of the Spark worker's token range
+     */
+    static ReplicaSet splitReplicas(@NotNull ConsistencyLevel consistencyLevel,
+                                    @Nullable String dataCenter,
+                                    @NotNull Map<Range<BigInteger>, List<CassandraInstance>> ranges,
+                                    @NotNull Set<CassandraInstance> replicas,
+                                    @NotNull Function<CassandraInstance, AvailabilityHint> availability,
+                                    int minReplicas,
+                                    int partitionId) throws NotEnoughReplicasException
+    {
+        ReplicaSet split = splitReplicas(replicas, ranges, availability, minReplicas, partitionId);
+        validateConsistency(consistencyLevel, dataCenter, ranges, split.primary(), minReplicas);
+        return split;
+    }
+
+    /**
+     * Validate we have achieved consistency for all sub-ranges owned by the Spark worker
+     *
+     * @param consistencyLevel consistency level
+     * @param dc               data center
+     * @param workerRanges     token sub-ranges owned by this Spark worker
+     * @param primaryReplicas  set of primary replicas selected
+     * @param minReplicas      minimum number of replicas required to meet consistency level
+     * @throws NotEnoughReplicasException thrown when insufficient primary replicas selected to achieve
+     *                                    consistency level for any sub-range of the Spark worker's token range
+     */
+    private static void validateConsistency(@NotNull ConsistencyLevel consistencyLevel,
+                                            @Nullable String dc,
+                                            @NotNull Map<Range<BigInteger>, List<CassandraInstance>> workerRanges,
+                                            @NotNull Set<CassandraInstance> primaryReplicas,
+                                            int minReplicas) throws NotEnoughReplicasException
+    {
+        for (Map.Entry<Range<BigInteger>, List<CassandraInstance>> range : workerRanges.entrySet())
+        {
+            int count = (int) range.getValue().stream().filter(primaryReplicas::contains).count();
+            if (count < minReplicas)
+            {
+                throw new NotEnoughReplicasException(consistencyLevel, range.getKey(), minReplicas, count, dc);
+            }
+        }
+    }
+
+    /**
+     * Return a set of primary and backup CassandraInstances to satisfy the consistency level.
+     *
+     * NOTE: This method current assumes that each Spark token worker owns a single replica set.
+     *
+     * @param instances    replicas that overlap with the Spark worker's token range
+     * @param ranges       all the token ranges owned by this Spark worker, and associated replicas
+     * @param availability availability hint provider for each CassandraInstance
+     * @param minReplicas  minimum number of replicas to achieve consistency
+     * @param partitionId  Spark worker partitionId
+     * @return a set of primary and backup replicas to read from
+     */
+    static ReplicaSet splitReplicas(Collection<CassandraInstance> instances,
+                                    @NotNull Map<Range<BigInteger>, List<CassandraInstance>> ranges,
+                                    Function<CassandraInstance, AvailabilityHint> availability,
+                                    int minReplicas,
+                                    int partitionId)
+    {
+        ReplicaSet replicaSet = new ReplicaSet(minReplicas, partitionId);
+
+        // Sort instances by status hint, so we attempt available instances first
+        // (e.g. we already know which instances are probably up from create snapshot request)
+        instances.stream()
+                 .sorted(Comparator.comparing(availability, AvailabilityHint.AVAILABILITY_HINT_COMPARATOR))
+                 .forEach(replicaSet::add);
+
+        if (ranges.size() != 1)
+        {
+            // Currently we don't support using incremental repair when Spark worker owns
+            // multiple replica sets but for current implementation of the TokenPartitioner
+            // it returns a single replica set per Spark worker/partition
+            LOGGER.warn("Cannot use incremental repair awareness when Spark partition owns more than one replica set, "
+                      + "performance will be degraded numRanges={}", ranges.size());
+            replicaSet.incrementalRepairPrimary = null;
+        }
+
+        return replicaSet;
+    }
+
+    public static class ReplicaSet
+    {
+        private final Set<CassandraInstance> primary;
+        private final Set<CassandraInstance> backup;
+        private final int minReplicas;
+        private final int partitionId;
+        private CassandraInstance incrementalRepairPrimary;
+
+        ReplicaSet(int minReplicas,
+                   int partitionId)
+        {
+            this.minReplicas = minReplicas;
+            this.partitionId = partitionId;
+            this.primary = new HashSet<>();
+            this.backup = new HashSet<>();
+        }
+
+        public ReplicaSet add(CassandraInstance instance)
+        {
+            if (primary.size() < minReplicas)
+            {
+                LOGGER.info("Selecting instance as primary replica nodeName={} token={} dc={} partitionId={}",
+                            instance.nodeName(), instance.token(), instance.dataCenter(), partitionId);
+                return addPrimary(instance);
+            }
+            return addBackup(instance);
+        }
+
+        public boolean isRepairPrimary(CassandraInstance instance)
+        {
+            return incrementalRepairPrimary == null || incrementalRepairPrimary.equals(instance);
+        }
+
+        public Set<CassandraInstance> primary()
+        {
+            return primary;
+        }
+
+        public ReplicaSet addPrimary(CassandraInstance instance)
+        {
+            if (incrementalRepairPrimary == null)
+            {
+                // Pick the first primary replica as a 'repair primary' to read repaired SSTables at CL ONE
+                incrementalRepairPrimary = instance;
+            }
+            primary.add(instance);
+            return this;
+        }
+
+        public Set<CassandraInstance> backup()
+        {
+            return backup;
+        }
+
+        public ReplicaSet addBackup(CassandraInstance instance)
+        {
+            LOGGER.info("Selecting instance as backup replica nodeName={} token={} dc={} partitionId={}",
+                        instance.nodeName(), instance.token(), instance.dataCenter(), partitionId);
+            backup.add(instance);
+            return this;
+        }
+    }
+
+    // CDC
+
+    public abstract CompletableFuture<List<CommitLog>> listCommitLogs(CassandraInstance instance);
+
+    @Override
+    public CommitLogProvider commitLogs(int partitionId)
+    {
+        return () -> {
+            TokenPartitioner tokenPartitioner = tokenPartitioner();
+            Range<BigInteger> range = tokenPartitioner.getTokenRange(partitionId);
+
+            // For CDC we read from all available replicas that overlap with Spark token range
+            Map<Range<BigInteger>, List<CassandraInstance>> subRanges = ring().getSubRanges(range).asMapOfRanges();
+            Set<CassandraInstance> replicas = subRanges.values().stream()
+                    .flatMap(Collection::stream)
+                    .filter(instance -> datacenter == null || instance.dataCenter().equalsIgnoreCase(datacenter))
+                    .collect(Collectors.toSet());
+            List<CompletableFuture<List<CommitLog>>> futures = replicas.stream()
+                    .map(this::listCommitLogs)
+                    .collect(Collectors.toList());
+
+            // Block to list replicas
+            List<List<CommitLog>> replicaLogs = FutureUtils.awaitAll(futures, true, throwable ->
+                    LOGGER.warn("Failed to list CDC commit logs on instance", ThrowableUtils.rootCause(throwable)));
+
+            int requiredReplicas = minimumReplicasForCdc();
+            if (replicaLogs.size() < requiredReplicas)
+            {
+                // We need *at least* local quorum for CDC to work, but if all nodes are up then read from LOCAL ALL
+                throw new NotEnoughReplicasException(
+                        String.format("CDC requires at least %d replicas but only %d responded",
+                                      requiredReplicas, replicaLogs.size()));
+            }
+
+            return replicaLogs.stream()
+                              .flatMap(Collection::stream);
+        };
+    }
+
+    public abstract ReplicationFactor replicationFactor(String keyspace);
+
+    @Override
+    public int minimumReplicasForCdc()
+    {
+        CassandraRing ring = ring();
+        ReplicationFactor replicationFactor = ring.replicationFactor();
+        validateReplicationFactor(replicationFactor);
+        return consistencyLevel.blockFor(replicationFactor, datacenter);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return new HashCodeBuilder()
+               .append(datacenter)
+               .toHashCode();
+    }
+
+    @Override
+    public boolean equals(Object other)
+    {
+        if (other == null)
+        {
+            return false;
+        }
+        if (this == other)
+        {
+            return true;
+        }
+        if (this.getClass() != other.getClass())
+        {
+            return false;
+        }
+
+        PartitionedDataLayer that = (PartitionedDataLayer) other;
+        return new EqualsBuilder()
+               .append(this.datacenter, that.datacenter)
+               .isEquals();
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/ReplicaSet.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/ReplicaSet.java
new file mode 100644
index 0000000..bf8d491
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/ReplicaSet.java
@@ -0,0 +1,92 @@
+/*
+ * 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.cassandra.spark.data;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.spark.data.partitioner.CassandraInstance;
+
+public class ReplicaSet
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(ReplicaSet.class);
+
+    private final Set<CassandraInstance> primary;
+    private final Set<CassandraInstance> backup;
+    private final int minReplicas;
+    private final int partitionId;
+    CassandraInstance incrementalRepairPrimary;
+
+    ReplicaSet(int minReplicas, int partitionId)
+    {
+        this.minReplicas = minReplicas;
+        this.partitionId = partitionId;
+        this.primary = new HashSet<>();
+        this.backup = new HashSet<>();
+    }
+
+    public ReplicaSet add(CassandraInstance instance)
+    {
+        if (primary.size() < minReplicas)
+        {
+            LOGGER.info("Selecting instance as primary replica nodeName={} token={} dataCenter={} partitionId={}",
+                        instance.nodeName(), instance.token(), instance.dataCenter(), partitionId);
+            return addPrimary(instance);
+        }
+        return addBackup(instance);
+    }
+
+    public boolean isRepairPrimary(CassandraInstance instance)
+    {
+        return incrementalRepairPrimary == null || incrementalRepairPrimary.equals(instance);
+    }
+
+    public Set<CassandraInstance> primary()
+    {
+        return primary;
+    }
+
+    public ReplicaSet addPrimary(CassandraInstance instance)
+    {
+        if (incrementalRepairPrimary == null)
+        {
+            // Pick the first primary replica as a 'repair primary' to read repaired SSTables at CL ONE
+            incrementalRepairPrimary = instance;
+        }
+        primary.add(instance);
+        return this;
+    }
+
+    public Set<CassandraInstance> backup()
+    {
+        return backup;
+    }
+
+    public ReplicaSet addBackup(CassandraInstance instance)
+    {
+        LOGGER.info("Selecting instance as backup replica nodeName={} token={} dataCenter={} partitionId={}",
+                    instance.nodeName(), instance.token(), instance.dataCenter(), partitionId);
+        backup.add(instance);
+        return this;
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/SidecarProvisionedSSTable.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/SidecarProvisionedSSTable.java
new file mode 100644
index 0000000..e985512
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/SidecarProvisionedSSTable.java
@@ -0,0 +1,262 @@
+/*
+ * 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.cassandra.spark.data;
+
+import java.io.ByteArrayInputStream;
+import java.io.InputStream;
+import java.time.Duration;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import org.apache.commons.io.IOUtils;
+
+import org.apache.cassandra.clients.Sidecar;
+import org.apache.cassandra.clients.SidecarStreamConsumerAdapter;
+import org.apache.cassandra.sidecar.client.SidecarClient;
+import org.apache.cassandra.sidecar.client.SidecarInstance;
+import org.apache.cassandra.sidecar.common.data.ListSnapshotFilesResponse;
+import org.apache.cassandra.sidecar.common.utils.HttpRange;
+import org.apache.cassandra.spark.stats.Stats;
+import org.apache.cassandra.spark.utils.ThrowableUtils;
+import org.apache.cassandra.spark.utils.streaming.SSTableInputStream;
+import org.apache.cassandra.spark.utils.streaming.SSTableSource;
+import org.apache.cassandra.spark.utils.streaming.StreamConsumer;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * An SSTable that is streamed from Sidecar
+ */
+public class SidecarProvisionedSSTable extends SSTable
+{
+    private static final long serialVersionUID = 6452703925812602832L;
+    private static final Cache<String, byte[]> COMPRESSION_CACHE = CacheBuilder.newBuilder()
+                                                                               .expireAfterAccess(1, TimeUnit.HOURS)
+                                                                               .maximumSize(2048)
+                                                                               .build();
+    private final SidecarClient sidecar;
+    private final SidecarInstance instance;
+    private final Sidecar.ClientConfig sidecarClientConfig;
+    private final String keyspace;
+    private final String table;
+    private final String snapshotName;
+    @NotNull
+    private final String dataFileName;
+    @NotNull
+    private final Map<FileType, ListSnapshotFilesResponse.FileInfo> components;
+    private final int partitionId;
+    private final Stats stats;
+
+    // CHECKSTYLE IGNORE: Constructor with many parameters
+    protected SidecarProvisionedSSTable(SidecarClient sidecar,
+                                        Sidecar.ClientConfig sidecarClientConfig,
+                                        SidecarInstance instance,
+                                        String keyspace,
+                                        String table,
+                                        String snapshotName,
+                                        @NotNull Map<FileType, ListSnapshotFilesResponse.FileInfo> components,
+                                        int partitionId,
+                                        Stats stats)
+    {
+        this.sidecar = sidecar;
+        this.sidecarClientConfig = sidecarClientConfig;
+        this.instance = instance;
+        this.keyspace = keyspace;
+        this.table = table;
+        this.snapshotName = snapshotName;
+        this.components = components;
+        this.partitionId = partitionId;
+        this.stats = stats;
+        String fileName = Objects.requireNonNull(components.get(FileType.DATA), "Data.db SSTable file component must exist").fileName;
+        String[] ssTableNameParts = fileName.split("-");
+        this.dataFileName = parseDataFileName(ssTableNameParts);
+    }
+
+    protected String parseDataFileName(String[] ssTableNameParts)
+    {
+        return String.join("-", ssTableNameParts[0], ssTableNameParts[1], ssTableNameParts[2], ssTableNameParts[3]);
+    }
+
+    public SidecarInstance instance()
+    {
+        return instance;
+    }
+
+    public int partitionId()
+    {
+        return partitionId;
+    }
+
+    @NotNull
+    @Override
+    public String getDataFileName()
+    {
+        return dataFileName;
+    }
+
+    @Nullable
+    @Override
+    protected InputStream openInputStream(FileType fileType)
+    {
+        ListSnapshotFilesResponse.FileInfo snapshotFile = components.get(fileType);
+        if (snapshotFile == null)
+        {
+            return null;
+        }
+        return openStream(snapshotFile.fileName, snapshotFile.size, fileType);
+    }
+
+    @Override
+    public boolean isMissing(FileType fileType)
+    {
+        return !components.containsKey(fileType);
+    }
+
+    @Nullable
+    private InputStream openStream(String component, long size, FileType fileType)
+    {
+        if (component == null)
+        {
+            return null;
+        }
+
+        if (fileType == FileType.COMPRESSION_INFO)
+        {
+            String key = String.format("%s/%s/%s/%s/%s", instance.hostname(), keyspace, table, snapshotName, component);
+            byte[] bytes;
+            try
+            {
+                bytes = COMPRESSION_CACHE.get(key, () -> IOUtils.toByteArray(open(component, fileType, size)));
+            }
+            catch (ExecutionException exception)
+            {
+                throw new RuntimeException(ThrowableUtils.rootCause(exception));
+            }
+            return new ByteArrayInputStream(bytes);
+        }
+
+        return open(component, fileType, size);
+    }
+
+    public InputStream open(String component, FileType fileType, long size)
+    {
+        SSTableSource<SidecarProvisionedSSTable> ssTableSource = source(component, fileType, size);
+        return new SSTableInputStream<>(ssTableSource, stats);
+    }
+
+    /**
+     * Build an SSTableSource to async provide the bytes
+     *
+     * @param componentName the SSTable component to stream
+     * @param fileType      SSTable file type
+     * @param size          file size in bytes
+     * @return an SSTableSource implementation that uses Sidecar client to request bytes
+     */
+    private SSTableSource<SidecarProvisionedSSTable> source(String componentName, FileType fileType, long size)
+    {
+        SidecarProvisionedSSTable thisSSTable = this;
+        return new SSTableSource<SidecarProvisionedSSTable>()
+        {
+            @Override
+            public void request(long start, long end, StreamConsumer consumer)
+            {
+                sidecar.streamSSTableComponent(instance,
+                                               keyspace,
+                                               table,
+                                               snapshotName,
+                                               componentName,
+                                               HttpRange.of(start, end),
+                                               new SidecarStreamConsumerAdapter(consumer));
+            }
+
+            @Override
+            public long maxBufferSize()
+            {
+                return sidecarClientConfig.maxBufferSize(fileType);
+            }
+
+            @Override
+            public long chunkBufferSize()
+            {
+                return sidecarClientConfig.chunkBufferSize(fileType);
+            }
+
+            @Nullable
+            @Override
+            public Duration timeout()
+            {
+                int timeout = sidecarClientConfig.timeoutSeconds();
+                return timeout > 0 ? Duration.ofSeconds(timeout) : null;
+            }
+
+            @Override
+            public SidecarProvisionedSSTable sstable()
+            {
+                return thisSSTable;
+            }
+
+            @Override
+            public FileType fileType()
+            {
+                return fileType;
+            }
+
+            @Override
+            public long size()
+            {
+                return size;
+            }
+        };
+    }
+
+    @Override
+    public String toString()
+    {
+        return String.format("{\"hostname\"=\"%s\", \"port\"=\"%d\", \"dataFileName\"=\"%s\", \"partitionId\"=\"%d\"}",
+                             instance.hostname(), instance.port(), dataFileName, partitionId);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Objects.hash(instance, dataFileName);
+    }
+
+    @Override
+    public boolean equals(Object other)
+    {
+        if (this == other)
+        {
+            return true;
+        }
+        if (other == null || this.getClass() != other.getClass())
+        {
+            return false;
+        }
+
+        SidecarProvisionedSSTable that = (SidecarProvisionedSSTable) other;
+        return this.instance.equals(that.instance)
+            && this.dataFileName.equals(that.dataFileName);
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/Sizing.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/Sizing.java
new file mode 100644
index 0000000..daca2d4
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/Sizing.java
@@ -0,0 +1,34 @@
+/*
+ * 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.cassandra.spark.data;
+
+/**
+ * Sizing options to determine the number of cores
+ */
+public interface Sizing
+{
+
+    /**
+     * Returns the effective number of executor cores to use in the spark application
+     *
+     * @return the effective number of executor cores to use in the spark application
+     */
+    int getEffectiveNumberOfCores();
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/partitioner/CassandraRing.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/partitioner/CassandraRing.java
new file mode 100644
index 0000000..2a3d0be
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/partitioner/CassandraRing.java
@@ -0,0 +1,339 @@
+/*
+ * 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.cassandra.spark.data.partitioner;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.Serializable;
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.TreeRangeMap;
+import org.apache.commons.lang.builder.EqualsBuilder;
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+import org.apache.cassandra.spark.data.ReplicationFactor;
+import org.apache.cassandra.spark.utils.RangeUtils;
+
+import static org.apache.cassandra.spark.data.ReplicationFactor.ReplicationStrategy.SimpleStrategy;
+
+/**
+ * CassandraRing is designed to have one unique way of handling
+ * Cassandra token/topology information across all Cassandra tooling.
+ * This class is made Serializable so it's easy to use it from Hadoop/Spark.
+ * As Cassandra token ranges are dependent on Replication strategy, ring makes sense for a specific keyspace only.
+ * It is made to be immutable for the sake of simplicity.
+ * <p>
+ * Token ranges are calculated assuming Cassandra racks are not being used, but controlled by assigning tokens properly.
+ * <p>
+ * {@link #equals(Object)} and {@link #hashCode()} don't take {@link #replicas} and {@link #tokenRangeMap}
+ * into consideration as they are just derived fields.
+ */
+@SuppressWarnings({"UnstableApiUsage", "unused", "WeakerAccess"})
+public class CassandraRing implements Serializable
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(CassandraRing.class);
+
+    private Partitioner partitioner;
+    private String keyspace;
+    private ReplicationFactor replicationFactor;
+    private ArrayList<CassandraInstance> instances;
+
+    private transient RangeMap<BigInteger, List<CassandraInstance>> replicas;
+    private transient Multimap<CassandraInstance, Range<BigInteger>> tokenRangeMap;
+
+    /**
+     * Add a replica with given range to replicaMap (RangeMap pointing to replicas).
+     * <p>
+     * replicaMap starts with full range (representing complete ring) with empty list of replicas. So, it is
+     * guaranteed that range will match one or many ranges in replicaMap.
+     * <p>
+     * Scheme to add a new replica for a range:
+     *   * Find overlapping rangeMap entries from replicaMap
+     *   * For each overlapping range, create new replica list by adding new replica to the existing list and add it
+     *     back to replicaMap.
+     */
+    private static void addReplica(CassandraInstance replica,
+                                   Range<BigInteger> range,
+                                   RangeMap<BigInteger, List<CassandraInstance>> replicaMap)
+    {
+        Preconditions.checkArgument(range.lowerEndpoint().compareTo(range.upperEndpoint()) <= 0,
+                                    "Range calculations assume range is not wrapped");
+
+        RangeMap<BigInteger, List<CassandraInstance>> replicaRanges = replicaMap.subRangeMap(range);
+        RangeMap<BigInteger, List<CassandraInstance>> mappingsToAdd = TreeRangeMap.create();
+
+        replicaRanges.asMapOfRanges().forEach((key, value) -> {
+            List<CassandraInstance> replicas = new ArrayList<>(value);
+            replicas.add(replica);
+            mappingsToAdd.put(key, replicas);
+        });
+        replicaMap.putAll(mappingsToAdd);
+    }
+
+    public CassandraRing(Partitioner partitioner,
+                         String keyspace,
+                         ReplicationFactor replicationFactor,
+                         Collection<CassandraInstance> instances)
+    {
+        this.partitioner = partitioner;
+        this.keyspace = keyspace;
+        this.replicationFactor = replicationFactor;
+        this.instances = instances.stream()
+                                  .sorted(Comparator.comparing(instance -> new BigInteger(instance.token())))
+                                  .collect(Collectors.toCollection(ArrayList::new));
+        this.init();
+    }
+
+    private void init()
+    {
+        // Setup token range map
+        replicas = TreeRangeMap.create();
+        tokenRangeMap = ArrayListMultimap.create();
+
+        // Calculate instance to token ranges mapping
+        switch (replicationFactor.getReplicationStrategy())
+        {
+            case SimpleStrategy:
+                tokenRangeMap.putAll(RangeUtils.calculateTokenRanges(instances,
+                                                                     replicationFactor.getTotalReplicationFactor(),
+                                                                     partitioner));
+                break;
+            case NetworkTopologyStrategy:
+                for (String dataCenter : dataCenters())
+                {
+                    int rf = replicationFactor.getOptions().get(dataCenter);
+                    if (rf == 0)
+                    {
+                        continue;
+                    }
+                    List<CassandraInstance> dcInstances = instances.stream()
+                            .filter(instance -> instance.dataCenter().matches(dataCenter))
+                            .collect(Collectors.toList());
+                    tokenRangeMap.putAll(RangeUtils.calculateTokenRanges(dcInstances,
+                                                                         replicationFactor.getOptions().get(dataCenter),
+                                                                         partitioner));
+                }
+                break;
+            default:
+                throw new UnsupportedOperationException("Unsupported replication strategy");
+        }
+
+        // Calculate token range to replica mapping
+        replicas.put(Range.closed(partitioner.minToken(), partitioner.maxToken()), Collections.emptyList());
+        tokenRangeMap.asMap().forEach((instance, ranges) -> ranges.forEach(range -> addReplica(instance, range, replicas)));
+    }
+
+    public Partitioner partitioner()
+    {
+        return partitioner;
+    }
+
+    public String keyspace()
+    {
+        return keyspace;
+    }
+
+    public Collection<CassandraInstance> instances()
+    {
+        return instances;
+    }
+
+    public Collection<CassandraInstance> getReplicas(BigInteger token)
+    {
+        return replicas.get(token);
+    }
+
+    public RangeMap<BigInteger, List<CassandraInstance>> rangeMap()
+    {
+        return replicas;
+    }
+
+    public ReplicationFactor replicationFactor()
+    {
+        return replicationFactor;
+    }
+
+    public RangeMap<BigInteger, List<CassandraInstance>> getSubRanges(Range<BigInteger> tokenRange)
+    {
+        return replicas.subRangeMap(tokenRange);
+    }
+
+    public Multimap<CassandraInstance, Range<BigInteger>> tokenRanges()
+    {
+        return tokenRangeMap;
+    }
+
+    private Collection<String> dataCenters()
+    {
+        return replicationFactor.getReplicationStrategy() == SimpleStrategy
+               ? Collections.emptySet()
+               : replicationFactor.getOptions().keySet();
+    }
+
+    public Collection<BigInteger> tokens()
+    {
+        return instances.stream()
+                        .map(CassandraInstance::token)
+                        .map(BigInteger::new)
+                        .sorted()
+                        .collect(Collectors.toList());
+    }
+
+    public Collection<BigInteger> tokens(String dataCenter)
+    {
+        Preconditions.checkArgument(replicationFactor.getReplicationStrategy() != SimpleStrategy,
+                                    "Datacenter tokens doesn't make sense for SimpleStrategy");
+        return instances.stream()
+                        .filter(instance -> instance.dataCenter().matches(dataCenter))
+                        .map(CassandraInstance::token)
+                        .map(BigInteger::new)
+                        .collect(Collectors.toList());
+    }
+
+    @Override
+    public boolean equals(Object other)
+    {
+        if (other == null)
+        {
+            return false;
+        }
+        if (this == other)
+        {
+            return true;
+        }
+        if (this.getClass() != other.getClass())
+        {
+            return false;
+        }
+
+        CassandraRing that = (CassandraRing) other;
+        return new EqualsBuilder()
+               .append(this.partitioner, that.partitioner)
+               .append(this.keyspace, that.keyspace)
+               .append(this.replicationFactor, that.replicationFactor)
+               .append(this.instances, that.instances)
+               .append(this.replicas, that.replicas)
+               .append(this.tokenRangeMap, that.tokenRangeMap)
+               .isEquals();
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return new HashCodeBuilder()
+               .append(partitioner)
+               .append(keyspace)
+               .append(replicationFactor)
+               .append(instances)
+               .append(replicas)
+               .append(tokenRangeMap)
+               .toHashCode();
+    }
+
+    private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException
+    {
+        LOGGER.warn("Falling back to JDK deserialization");
+        this.partitioner = in.readByte() == 0 ? Partitioner.RandomPartitioner : Partitioner.Murmur3Partitioner;
+        this.keyspace = in.readUTF();
+
+        ReplicationFactor.ReplicationStrategy strategy = ReplicationFactor.ReplicationStrategy.valueOf(in.readByte());
+        int optionCount = in.readByte();
+        Map<String, Integer> options = new HashMap<>(optionCount);
+        for (int option = 0; option < optionCount; option++)
+        {
+            options.put(in.readUTF(), (int) in.readByte());
+        }
+        this.replicationFactor = new ReplicationFactor(strategy, options);
+
+        int numInstances = in.readShort();
+        this.instances = new ArrayList<>(numInstances);
+        for (int instance = 0; instance < numInstances; instance++)
+        {
+            this.instances.add(new CassandraInstance(in.readUTF(), in.readUTF(), in.readUTF()));
+        }
+        this.init();
+    }
+
+    private void writeObject(ObjectOutputStream out) throws IOException, ClassNotFoundException
+    {
+        LOGGER.warn("Falling back to JDK serialization");
+        out.writeByte(this.partitioner == Partitioner.RandomPartitioner ? 0 : 1);
+        out.writeUTF(this.keyspace);
+
+        out.writeByte(this.replicationFactor.getReplicationStrategy().value);
+        Map<String, Integer> options = this.replicationFactor.getOptions();
+        out.writeByte(options.size());
+        for (Map.Entry<String, Integer> option : options.entrySet())
+        {
+            out.writeUTF(option.getKey());
+            out.writeByte(option.getValue());
+        }
+
+        out.writeShort(this.instances.size());
+        for (CassandraInstance instance : this.instances)
+        {
+            out.writeUTF(instance.token());
+            out.writeUTF(instance.nodeName());
+            out.writeUTF(instance.dataCenter());
+        }
+    }
+
+    public static class Serializer extends com.esotericsoftware.kryo.Serializer<CassandraRing>
+    {
+        @Override
+        public void write(Kryo kryo, Output out, CassandraRing ring)
+        {
+            out.writeByte(ring.partitioner == Partitioner.RandomPartitioner ? 1 : 0);
+            out.writeString(ring.keyspace);
+            kryo.writeObject(out, ring.replicationFactor);
+            kryo.writeObject(out, ring.instances);
+        }
+
+        @Override
+        @SuppressWarnings("unchecked")
+        public CassandraRing read(Kryo kryo, Input in, Class<CassandraRing> type)
+        {
+            return new CassandraRing(in.readByte() == 1 ? Partitioner.RandomPartitioner
+                                                        : Partitioner.Murmur3Partitioner,
+                                     in.readString(),
+                                     kryo.readObject(in, ReplicationFactor.class),
+                                     kryo.readObject(in, ArrayList.class));
+        }
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/partitioner/ConsistencyLevel.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/partitioner/ConsistencyLevel.java
new file mode 100644
index 0000000..529d9fd
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/partitioner/ConsistencyLevel.java
@@ -0,0 +1,124 @@
+/*
+ * 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.cassandra.spark.data.partitioner;
+
+import org.apache.cassandra.spark.data.ReplicationFactor;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+public enum ConsistencyLevel
+{
+    ANY(0),
+    ONE(1),
+    TWO(2),
+    THREE(3),
+    QUORUM(4),
+    ALL(5),
+    LOCAL_QUORUM(6, true),
+    EACH_QUORUM(7),
+    SERIAL(8),
+    LOCAL_SERIAL(9),
+    LOCAL_ONE(10, true);
+
+    public final int code;
+    public final boolean isDCLocal;
+
+    ConsistencyLevel(int code)
+    {
+        this(code, false);
+    }
+
+    ConsistencyLevel(int code, boolean isDCLocal)
+    {
+        this.code = code;
+        this.isDCLocal = isDCLocal;
+    }
+
+    private int quorumFor(ReplicationFactor replicationFactor)
+    {
+        return (replicationFactor.getTotalReplicationFactor() / 2) + 1;
+    }
+
+    private int localQuorumFor(@NotNull ReplicationFactor replicationFactor, @Nullable String dataCenter)
+    {
+        return replicationFactor.getReplicationStrategy() == ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy
+               ? getNetworkTopologyRf(replicationFactor, dataCenter)
+               : quorumFor(replicationFactor);
+    }
+
+    private int getNetworkTopologyRf(@NotNull ReplicationFactor replicationFactor, @Nullable String dataCenter)
+    {
+        int rf;
+        // Single data center and not specified, so return the only data center replication factor
+        if (dataCenter == null && replicationFactor.getOptions().size() == 1)
+        {
+            rf = replicationFactor.getOptions().values().iterator().next();
+        }
+        else
+        {
+            if (!replicationFactor.getOptions().containsKey(dataCenter))
+            {
+                throw new IllegalArgumentException(String.format("Data center %s not found in replication factor %s",
+                                                                 dataCenter, replicationFactor.getOptions().keySet()));
+            }
+            rf = replicationFactor.getOptions().get(dataCenter);
+        }
+        return (rf / 2) + 1;
+    }
+
+    public int blockFor(@NotNull ReplicationFactor replicationFactor, @Nullable String dataCenter)
+    {
+        switch (this)
+        {
+            case ONE:
+            case LOCAL_ONE:
+            case ANY:
+                return 1;
+            case TWO:
+                return 2;
+            case THREE:
+                return 3;
+            case QUORUM:
+            case SERIAL:
+                return quorumFor(replicationFactor);
+            case ALL:
+                return replicationFactor.getTotalReplicationFactor();
+            case LOCAL_QUORUM:
+            case LOCAL_SERIAL:
+                return localQuorumFor(replicationFactor, dataCenter);
+            case EACH_QUORUM:
+                if (replicationFactor.getReplicationStrategy() == ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy)
+                {
+                    int count = 0;
+                    for (String datacenter : replicationFactor.getOptions().keySet())
+                    {
+                        count += localQuorumFor(replicationFactor, datacenter);
+                    }
+                    return count;
+                }
+                else
+                {
+                    return quorumFor(replicationFactor);
+                }
+            default:
+                throw new UnsupportedOperationException("Invalid consistency level: " + toString());
+        }
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/partitioner/MultipleReplicas.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/partitioner/MultipleReplicas.java
new file mode 100644
index 0000000..12894b3
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/partitioner/MultipleReplicas.java
@@ -0,0 +1,153 @@
+/*
+ * 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.cassandra.spark.data.partitioner;
+
+import java.util.Collections;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.collect.ImmutableSet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.spark.data.SSTablesSupplier;
+import org.apache.cassandra.spark.reader.SparkSSTableReader;
+import org.apache.cassandra.spark.stats.Stats;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Return a set of SSTables for a token range, returning enough replica copies to satisfy consistency level
+ */
+public class MultipleReplicas extends SSTablesSupplier
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(MultipleReplicas.class);
+
+    @NotNull
+    private final Set<SingleReplica> primaryReplicas;
+    @NotNull
+    private final Set<SingleReplica> backupReplicas;
+    @NotNull
+    private final Stats stats;
+
+    public MultipleReplicas(@NotNull Set<SingleReplica> primaryReplicas,
+                            @NotNull Set<SingleReplica> backupReplicas,
+                            @NotNull Stats stats)
+    {
+        this.primaryReplicas = ImmutableSet.copyOf(primaryReplicas);
+        this.backupReplicas = ImmutableSet.copyOf(backupReplicas);
+        this.stats = stats;
+    }
+
+    /**
+     * Open SSTable readers for enough replicas to satisfy consistency level
+     *
+     * @param readerOpener open SparkSSTableReader for SSTable
+     * @return set of SparkSSTableReaders to compact
+     */
+    @Override
+    public <T extends SparkSSTableReader> Set<T> openAll(ReaderOpener<T> readerOpener)
+    {
+        if (primaryReplicas.isEmpty())
+        {
+            return Collections.emptySet();
+        }
+
+        long startTimeNanos = System.nanoTime();
+        ConcurrentLinkedQueue<SingleReplica> otherReplicas = new ConcurrentLinkedQueue<>(backupReplicas);
+        AtomicInteger count = new AtomicInteger(0);
+        CountDownLatch latch = new CountDownLatch(primaryReplicas.size());
+        Set<T> result = ConcurrentHashMap.newKeySet();
+        // Open all primary replicas async
+        for (SingleReplica primaryReplica : primaryReplicas)
+        {
+            openReplicaOrRetry(primaryReplica, readerOpener, result, count, latch, otherReplicas);
+        }
+
+        // Block until all replicas opened
+        try
+        {
+            latch.await();
+        }
+        catch (InterruptedException exception)
+        {
+            Thread.currentThread().interrupt();
+            throw new RuntimeException(exception);
+        }
+
+        // Need to meet the required number of primary replicas to meet consistency level
+        if (count.get() < primaryReplicas.size())
+        {
+            stats.notEnoughReplicas(primaryReplicas, backupReplicas);
+            throw new NotEnoughReplicasException(String.format("Required %d replicas but only %d responded",
+                                                               primaryReplicas.size(), count.get()));
+        }
+
+        stats.openedReplicas(primaryReplicas, backupReplicas, System.nanoTime() - startTimeNanos);
+        return ImmutableSet.copyOf(result);
+    }
+
+    private <T extends SparkSSTableReader> void openReplicaOrRetry(
+            @NotNull SingleReplica replica,
+            @NotNull ReaderOpener<T> readerOpener,
+            @NotNull Set<T> result,
+            @NotNull AtomicInteger count,
+            @NotNull CountDownLatch latch,
+            @NotNull ConcurrentLinkedQueue<SingleReplica> otherReplicas)
+    {
+        replica.openReplicaAsync(readerOpener)
+               .whenComplete((readers, throwable) -> {
+                   if (throwable != null)
+                   {
+                       LOGGER.warn("Failed to open SSTableReaders for replica node={} token={} dataCenter={}",
+                                   replica.instance().nodeName(), replica.instance().token(), replica.instance().dataCenter(), throwable);
+                       stats.failedToOpenReplica(replica, throwable);
+                       SingleReplica anotherReplica = otherReplicas.poll();
+                       if (anotherReplica != null)
+                       {
+                           LOGGER.warn("Retrying on another replica node={} token={} dataCenter={}",
+                                       anotherReplica.instance().nodeName(), anotherReplica.instance().token(), anotherReplica.instance().dataCenter());
+                           // If the failed replica was the repair primary we need the backup replacement replica to be the new repair primary
+                           anotherReplica.setIsRepairPrimary(replica.isRepairPrimary());
+                           openReplicaOrRetry(anotherReplica, readerOpener, result, count, latch, otherReplicas);
+                       }
+                       else
+                       // No more replicas to retry so end
+                       {
+                           latch.countDown();
+                       }
+                       return;
+                   }
+
+                   try
+                   {
+                       // Successfully opened all SSTable readers
+                       result.addAll(readers);
+                       count.incrementAndGet();
+                   }
+                   finally
+                   {
+                       latch.countDown();
+                   }
+               });
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/partitioner/NotEnoughReplicasException.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/partitioner/NotEnoughReplicasException.java
new file mode 100644
index 0000000..7eab83b
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/partitioner/NotEnoughReplicasException.java
@@ -0,0 +1,59 @@
+/*
+ * 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.cassandra.spark.data.partitioner;
+
+import java.math.BigInteger;
+import javax.validation.constraints.NotNull;
+
+import com.google.common.collect.Range;
+
+import org.jetbrains.annotations.Nullable;
+
+public class NotEnoughReplicasException extends RuntimeException
+{
+    public NotEnoughReplicasException(String message)
+    {
+        super(message);
+    }
+
+    public NotEnoughReplicasException(@NotNull ConsistencyLevel consistencyLevel,
+                                      @NotNull Range<BigInteger> range,
+                                      int minRequired,
+                                      int numInstances,
+                                      @Nullable String dataCenter)
+    {
+        super(String.format("Insufficient replicas found to achieve consistency level %s for token range %s - %s, "
+                          + "required %d but only %d found, dataCenter=%s",
+                            consistencyLevel.name(), range.lowerEndpoint(), range.upperEndpoint(),
+                            minRequired, numInstances, dataCenter));
+    }
+
+    static boolean isNotEnoughReplicasException(@Nullable Throwable throwable)
+    {
+        for (Throwable cause = throwable; cause != null; cause = cause.getCause())
+        {
+            if (cause instanceof NotEnoughReplicasException)
+            {
+                return true;
+            }
+        }
+        return false;
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/partitioner/SingleReplica.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/partitioner/SingleReplica.java
new file mode 100644
index 0000000..0454400
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/partitioner/SingleReplica.java
@@ -0,0 +1,186 @@
+/*
+ * 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.cassandra.spark.data.partitioner;
+
+import java.io.IOException;
+import java.math.BigInteger;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.stream.Stream;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Range;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.spark.data.PartitionedDataLayer;
+import org.apache.cassandra.spark.data.SSTable;
+import org.apache.cassandra.spark.data.SSTablesSupplier;
+import org.apache.cassandra.spark.reader.SparkSSTableReader;
+import org.apache.cassandra.spark.reader.common.SSTableStreamException;
+import org.apache.cassandra.spark.stats.Stats;
+import org.apache.cassandra.spark.utils.ThrowableUtils;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Return a set of SSTables for a single Cassandra Instance
+ */
+public class SingleReplica extends SSTablesSupplier
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(SingleReplica.class);
+
+    private final PartitionedDataLayer dataLayer;
+    private final CassandraInstance instance;
+    private final Range<BigInteger> range;
+    private final int partitionId;
+    private final ExecutorService executor;
+    private final Stats stats;
+    private boolean isRepairPrimary;
+
+    public SingleReplica(@NotNull CassandraInstance instance,
+                         @NotNull PartitionedDataLayer dataLayer,
+                         @NotNull Range<BigInteger> range,
+                         int partitionId,
+                         @NotNull ExecutorService executor,
+                         boolean isRepairPrimary)
+    {
+        this(instance, dataLayer, range, partitionId, executor, Stats.DoNothingStats.INSTANCE, isRepairPrimary);
+    }
+
+    public SingleReplica(@NotNull CassandraInstance instance,
+                         @NotNull PartitionedDataLayer dataLayer,
+                         @NotNull Range<BigInteger> range,
+                         int partitionId,
+                         @NotNull ExecutorService executor,
+                         @NotNull Stats stats,
+                         boolean isRepairPrimary)
+    {
+        this.dataLayer = dataLayer;
+        this.instance = instance;
+        this.range = range;
+        this.partitionId = partitionId;
+        this.executor = executor;
+        this.stats = stats;
+        this.isRepairPrimary = isRepairPrimary;
+    }
+
+    public CassandraInstance instance()
+    {
+        return instance;
+    }
+
+    public Range<BigInteger> range()
+    {
+        return range;
+    }
+
+    public boolean isRepairPrimary()
+    {
+        return isRepairPrimary;
+    }
+
+    public void setIsRepairPrimary(boolean isRepairPrimary)
+    {
+        this.isRepairPrimary = isRepairPrimary;
+    }
+
+    /**
+     * Open all SparkSSTableReaders for all SSTables for this replica
+     *
+     * @param readerOpener provides function to open SparkSSTableReader using SSTable
+     * @return set of SparkSSTableReader to pass over to the CompactionIterator
+     */
+    @Override
+    public <T extends SparkSSTableReader> Set<T> openAll(ReaderOpener<T> readerOpener)
+    {
+        try
+        {
+            return openReplicaAsync(readerOpener).get();
+        }
+        catch (InterruptedException exception)
+        {
+            Thread.currentThread().interrupt();
+            throw new RuntimeException(exception);
+        }
+        catch (ExecutionException exception)
+        {
+            throw new RuntimeException(ThrowableUtils.rootCause(exception));
+        }
+    }
+
+    <T extends SparkSSTableReader> CompletableFuture<Set<T>> openReplicaAsync(@NotNull ReaderOpener<T> readerOpener)
+    {
+        // List SSTables and open SSTable readers
+        try
+        {
+            long timeNanos = System.nanoTime();
+            return dataLayer.listInstance(partitionId, range, instance)
+                            .thenApply(stream -> {
+                                stats.timeToListSnapshot(this, System.nanoTime() - timeNanos);
+                                return stream;
+                            })
+                            .thenCompose(stream -> openAll(stream, readerOpener));
+        }
+        catch (Throwable throwable)
+        {
+            LOGGER.warn("Unexpected error attempting to open SSTable readers for replica node={} token={} dataCenter={}",
+                        instance().nodeName(), instance().token(), instance().dataCenter(), throwable);
+            CompletableFuture<Set<T>> exceptionally = new CompletableFuture<>();
+            exceptionally.completeExceptionally(throwable);
+            return exceptionally;
+        }
+    }
+
+    private <T extends SparkSSTableReader> CompletableFuture<Set<T>> openAll(@NotNull Stream<SSTable> stream,
+                                                                             @NotNull ReaderOpener<T> readerOpener)
+    {
+        Set<T> result = ConcurrentHashMap.newKeySet();
+        CompletableFuture[] futures = stream
+                // Verify all the required SSTable file components are available
+                .peek(SSTable::verify)
+                // Open SSTable readers in parallel using executor
+                .map(ssTable -> CompletableFuture.runAsync(() -> openReader(readerOpener, ssTable, result), executor))
+                .toArray(CompletableFuture[]::new);
+
+        // All futures must complete non-exceptionally for the resulting future to complete
+        return CompletableFuture.allOf(futures).thenApply(aVoid -> ImmutableSet.copyOf(result));
+    }
+
+    private <T extends SparkSSTableReader> void openReader(@NotNull ReaderOpener<T> readerOpener,
+                                                           @NotNull SSTable ssTable,
+                                                           @NotNull Set<T> result)
+    {
+        try
+        {
+            T reader = readerOpener.openReader(ssTable, isRepairPrimary);
+            if (!reader.ignore())
+            {
+                result.add(reader);
+            }
+        }
+        catch (IOException exception)
+        {
+            throw new SSTableStreamException(exception);
+        }
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/partitioner/TokenPartitioner.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/partitioner/TokenPartitioner.java
new file mode 100644
index 0000000..140e1e3
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/data/partitioner/TokenPartitioner.java
@@ -0,0 +1,289 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.cassandra.spark.data.partitioner;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.Serializable;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.BoundType;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeMap;
+import com.google.common.collect.RangeSet;
+import com.google.common.collect.TreeRangeMap;
+import com.google.common.collect.TreeRangeSet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+import org.apache.cassandra.spark.utils.ByteBufferUtils;
+import org.apache.cassandra.spark.utils.RangeUtils;
+
+/**
+ * Util class for partitioning Spark workers across the token ring
+ */
+@SuppressWarnings("UnstableApiUsage")
+public class TokenPartitioner implements Serializable
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(TokenPartitioner.class);
+
+    private List<Range<BigInteger>> subRanges;
+    private CassandraRing ring;
+    private transient RangeMap<BigInteger, Integer> partitionMap;
+    private transient Map<Integer, Range<BigInteger>> reversePartitionMap;
+
+    protected TokenPartitioner(List<Range<BigInteger>> subRanges, CassandraRing ring)
+    {
+        this.subRanges = subRanges;
+        this.ring = ring;
+        this.partitionMap = TreeRangeMap.create();
+        this.reversePartitionMap = new HashMap<>();
+        calculateTokenRangeMap();
+    }
+
+    public TokenPartitioner(CassandraRing ring, int defaultParallelism, int numCores)
+    {
+        this(ring, defaultParallelism, numCores, false);
+    }
+
+    public TokenPartitioner(CassandraRing ring, int defaultParallelism, int numCores, boolean shuffle)
+    {
+        LOGGER.info("Creating TokenPartitioner defaultParallelism={} numCores={}", defaultParallelism, numCores);
+        this.partitionMap = TreeRangeMap.create();
+        this.reversePartitionMap = new HashMap<>();
+        this.ring = ring;
+
+        int numSplits = TokenPartitioner.calculateSplits(ring, defaultParallelism, numCores);
+        this.subRanges = ring.rangeMap().asMapOfRanges().keySet().stream()
+                             .flatMap(tr -> RangeUtils.split(tr, numSplits).stream()).collect(Collectors.toList());
+
+        // Shuffle off by default to avoid every spark worker connecting to every Cassandra instance
+        if (shuffle)
+        {
+            // Spark executes workers in partition order so here we shuffle the sub-ranges before
+            // assigning to a Spark partition so the job executes more evenly across the token ring
+            Collections.shuffle(subRanges);
+        }
+
+        calculateTokenRangeMap();
+    }
+
+    private void calculateTokenRangeMap()
+    {
+        int nextPartitionId = 0;
+        for (Range<BigInteger> tr : subRanges)
+        {
+            int partitionId = nextPartitionId;
+            partitionMap.put(tr, partitionId);
+            reversePartitionMap.put(partitionId, tr);
+            nextPartitionId++;
+        }
+
+        validateMapSizes();
+        validateCompleteRangeCoverage();
+        validateRangesDoNotOverlap();
+
+        LOGGER.info("Number of partitions {}", reversePartitionMap.size());
+        LOGGER.info("Partition map " + partitionMap);
+        LOGGER.info("Reverse partition map " + reversePartitionMap);
+    }
+
+    private static int calculateSplits(CassandraRing ring, int defaultParallelism, Integer cores)
+    {
+        int tasksToRun = Math.max(cores, defaultParallelism);
+        LOGGER.info("Tasks to run: {}", tasksToRun);
+        Map<Range<BigInteger>, List<CassandraInstance>> rangeListMap = ring.rangeMap().asMapOfRanges();
+        LOGGER.info("Initial ranges: {}", rangeListMap);
+        int ranges = rangeListMap.size();
+        LOGGER.info("Number of ranges: {}", ranges);
+        int calculatedSplits = TokenPartitioner.divCeil(tasksToRun, ranges);
+        LOGGER.info("Calculated number of splits as {}", calculatedSplits);
+        return calculatedSplits;
+    }
+
+    public CassandraRing ring()
+    {
+        return ring;
+    }
+
+    public List<Range<BigInteger>> subRanges()
+    {
+        return subRanges;
+    }
+
+    public RangeMap<BigInteger, Integer> partitionMap()
+    {
+        return partitionMap;
+    }
+
+    public Map<Integer, Range<BigInteger>> reversePartitionMap()
+    {
+        return reversePartitionMap;
+    }
+
+    private static int divCeil(int a, int b)
+    {
+        return (a + b - 1) / b;
+    }
+
+    public int numPartitions()
+    {
+        return reversePartitionMap.size();
+    }
+
+    @SuppressWarnings("ConstantConditions")
+    public boolean isInPartition(BigInteger token, ByteBuffer key, int partitionId)
+    {
+        boolean isInPartition = partitionId == partitionMap.get(token);
+        if (LOGGER.isDebugEnabled() && !isInPartition)
+        {
+            Range<BigInteger> range = getTokenRange(partitionId);
+            LOGGER.debug("Filtering out partition key key='{}' token={} rangeLower={} rangeUpper={}",
+                         ByteBufferUtils.toHexString(key), token, range.lowerEndpoint(), range.upperEndpoint());
+        }
+        return isInPartition;
+    }
+
+    public Range<BigInteger> getTokenRange(int partitionId)
+    {
+        return reversePartitionMap.get(partitionId);
+    }
+
+    // Validation
+
+    private void validateRangesDoNotOverlap()
+    {
+        List<Range<BigInteger>> sortedRanges = partitionMap.asMapOfRanges().keySet().stream()
+                .sorted(Comparator.comparing(Range::lowerEndpoint))
+                .collect(Collectors.toList());
+        Range<BigInteger> previous = null;
+        for (Range<BigInteger> current : sortedRanges)
+        {
+            if (previous != null)
+            {
+                Preconditions.checkState(!current.isConnected(previous) || current.intersection(previous).isEmpty(),
+                                         String.format("Two ranges in partition map are overlapping %s %s",
+                                                       previous, current));
+            }
+
+            previous = current;
+        }
+    }
+
+    private void validateCompleteRangeCoverage()
+    {
+        RangeSet<BigInteger> missingRangeSet = TreeRangeSet.create();
+        missingRangeSet.add(Range.closed(ring.partitioner().minToken(),
+                                         ring.partitioner().maxToken()));
+
+        partitionMap.asMapOfRanges().keySet().forEach(missingRangeSet::remove);
+
+        List<Range<BigInteger>> missingRanges = missingRangeSet.asRanges().stream()
+                                                                          .filter(Range::isEmpty)
+                                                                          .collect(Collectors.toList());
+        Preconditions.checkState(missingRanges.isEmpty(),
+                                 "There should be no missing ranges, but found " + missingRanges.toString());
+    }
+
+    private void validateMapSizes()
+    {
+        int nrPartitions = numPartitions();
+        Preconditions.checkState(nrPartitions == partitionMap.asMapOfRanges().keySet().size(),
+                                 String.format("Number of partitions %d not matching with partition map size %d",
+                                               nrPartitions, partitionMap.asMapOfRanges().keySet().size()));
+        Preconditions.checkState(nrPartitions == reversePartitionMap.keySet().size(),
+                                 String.format("Number of partitions %d not matching with reverse partition map size %d",
+                                               nrPartitions, reversePartitionMap.keySet().size()));
+        Preconditions.checkState(nrPartitions >= ring.rangeMap().asMapOfRanges().keySet().size(),
+                                 String.format("Number of partitions %d supposed to be more than number of token ranges %d",
+                                               nrPartitions, ring.rangeMap().asMapOfRanges().keySet().size()));
+        Preconditions.checkState(nrPartitions >= ring.tokenRanges().keySet().size(),
+                                 String.format("Number of partitions %d supposed to be more than number of instances %d",
+                                               nrPartitions, ring.tokenRanges().keySet().size()));
+        Preconditions.checkState(partitionMap.asMapOfRanges().keySet().size() == reversePartitionMap.keySet().size(),
+                                 String.format("You must be kidding me! Partition map %d and reverse map %d are not of same size",
+                                               partitionMap.asMapOfRanges().keySet().size(),
+                                               reversePartitionMap.keySet().size()));
+    }
+
+    @SuppressWarnings("unchecked")
+    private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException
+    {
+        LOGGER.warn("Falling back to JDK deserialization");
+        this.partitionMap = TreeRangeMap.create();
+        this.reversePartitionMap = new HashMap<>();
+        this.ring = (CassandraRing) in.readObject();
+        this.subRanges = (List<Range<BigInteger>>) in.readObject();
+        this.calculateTokenRangeMap();
+    }
+
+    private void writeObject(ObjectOutputStream out) throws IOException, ClassNotFoundException
+    {
+        LOGGER.warn("Falling back to JDK serialization");
+        out.writeObject(this.ring);
+        out.writeObject(this.subRanges);
+    }
+
+    public static class Serializer extends com.esotericsoftware.kryo.Serializer<TokenPartitioner>
+    {
+        @Override
+        public void write(Kryo kryo, Output out, TokenPartitioner partitioner)
+        {
+            out.writeInt(partitioner.subRanges.size());
+            for (Range<BigInteger> subRange : partitioner.subRanges)
+            {
+                out.writeByte(subRange.lowerBoundType() == BoundType.OPEN ? 1 : 0);
+                out.writeString(subRange.lowerEndpoint().toString());
+                out.writeByte(subRange.upperBoundType() == BoundType.OPEN ? 1 : 0);
+                out.writeString(subRange.upperEndpoint().toString());
+            }
+            kryo.writeObject(out, partitioner.ring);
+        }
+
+        @Override
+        public TokenPartitioner read(Kryo kryo, Input in, Class<TokenPartitioner> type)
+        {
+            int numRanges = in.readInt();
+            List<Range<BigInteger>> subRanges = new ArrayList<>(numRanges);
+            for (int range = 0; range < numRanges; range++)
+            {
+                BoundType lowerBoundType = in.readByte() == 1 ? BoundType.OPEN : BoundType.CLOSED;
+                BigInteger lowerBound = new BigInteger(in.readString());
+                BoundType upperBoundType = in.readByte() == 1 ? BoundType.OPEN : BoundType.CLOSED;
+                BigInteger upperBound = new BigInteger(in.readString());
+                subRanges.add(Range.range(lowerBound, lowerBoundType, upperBound, upperBoundType));
+            }
+            return new TokenPartitioner(subRanges, kryo.readObject(in, CassandraRing.class));
+        }
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/sparksql/AbstractSparkRowIterator.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/sparksql/AbstractSparkRowIterator.java
new file mode 100644
index 0000000..a07c3cb
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/sparksql/AbstractSparkRowIterator.java
@@ -0,0 +1,160 @@
+/*
+ * 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.cassandra.spark.sparksql;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.cassandra.spark.config.SchemaFeature;
+import org.apache.cassandra.spark.data.CqlField;
+import org.apache.cassandra.spark.data.CqlTable;
+import org.apache.cassandra.spark.data.DataLayer;
+import org.apache.cassandra.spark.sparksql.filters.CdcOffsetFilter;
+import org.apache.cassandra.spark.sparksql.filters.PartitionKeyFilter;
+import org.apache.cassandra.spark.stats.Stats;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Wrapper iterator around SparkCellIterator to normalize cells into Spark SQL rows
+ */
+abstract class AbstractSparkRowIterator
+{
+    private final Stats stats;
+    private final SparkCellIterator it;
+    private final long openTimeNanos;
+    private final RowBuilder builder;
+
+    protected final List<SchemaFeature> requestedFeatures;
+    protected final CqlTable cqlTable;
+    protected final boolean noValueColumns;
+    protected final StructType columnFilter;
+
+    private Cell cell = null;
+    private InternalRow row = null;
+
+    AbstractSparkRowIterator(int partitionId,
+                             @NotNull DataLayer dataLayer,
+                             @Nullable StructType requiredSchema,
+                             @NotNull List<PartitionKeyFilter> partitionKeyFilters,
+                             @Nullable CdcOffsetFilter cdcOffsetFilter)
+    {
+        this.stats = dataLayer.stats();
+        this.cqlTable = dataLayer.cqlTable();
+        this.columnFilter = useColumnFilter(requiredSchema, cqlTable) ? requiredSchema : null;
+        this.it = buildCellIterator(partitionId, dataLayer, columnFilter, partitionKeyFilters, cdcOffsetFilter);
+        this.stats.openedSparkRowIterator();
+        this.openTimeNanos = System.nanoTime();
+        this.requestedFeatures = dataLayer.requestedFeatures();
+        this.noValueColumns = it.noValueColumns();
+        this.builder = newBuilder();
+    }
+
+    protected SparkCellIterator buildCellIterator(int partitionId,
+                                                  @NotNull DataLayer dataLayer,
+                                                  @Nullable StructType columnFilter,
+                                                  @NotNull List<PartitionKeyFilter> partitionKeyFilters,
+                                                  @Nullable CdcOffsetFilter cdcOffsetFilter)
+    {
+        return new SparkCellIterator(partitionId, dataLayer, this.columnFilter, partitionKeyFilters, cdcOffsetFilter);  // TODO: This 'this.' feels wrong here
+    }
+
+    private static boolean useColumnFilter(@Nullable StructType requiredSchema, CqlTable cqlTable)
+    {
+        if (requiredSchema == null)
+        {
+            return false;
+        }
+        // Only use column filter if it excludes any of the CqlTable fields
+        Set<String> requiredFields = Arrays.stream(requiredSchema.fields()).map(StructField::name).collect(Collectors.toSet());
+        return cqlTable.fields().stream()
+                       .map(CqlField::name)
+                       .anyMatch(field -> !requiredFields.contains(field));
+    }
+
+    abstract RowBuilder newBuilder();
+
+    public InternalRow get()
+    {
+        return row;
+    }
+
+    public boolean next() throws IOException
+    {
+        // We are finished if not already reading a row (if cell != null, it can happen if previous row was incomplete)
+        // and SparkCellIterator has no next value
+        if (cell == null && !it.hasNextThrows())
+        {
+            return false;
+        }
+
+        // Pivot values to normalize each cell into single SparkSQL or 'CQL' type row
+        do
+        {
+            if (cell == null)
+            {
+                // Read next cell
+                cell = it.next();
+            }
+
+            if (builder.isFirstCell())
+            {
+                // On first iteration, copy all partition keys, clustering keys, static columns
+                assert cell.isNewRow;
+                builder.copyKeys(cell);
+            }
+            else if (cell.isNewRow)
+            {
+                // Current row is incomplete, so we have moved to new row before reaching end
+                // break out to return current incomplete row and handle next row in next iteration
+                break;
+            }
+
+            builder.onCell(cell);
+
+            if (!noValueColumns && !cell.isTombstone())
+            {
+                // If schema has value column or not a row deletion then copy across
+                builder.copyValue(cell);
+            }
+            cell = null;
+            // Keep reading more cells until we read the entire row
+        } while (builder.hasMoreCells() && it.hasNextThrows());
+
+        // Build row and reset builder for next row
+        row = builder.build();
+        builder.reset();
+
+        stats.nextRow();
+        return true;
+    }
+
+    public void close() throws IOException
+    {
+        stats.closedSparkRowIterator(System.nanoTime() - openTimeNanos);
+        it.close();
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/sparksql/CassandraDataSink.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/sparksql/CassandraDataSink.java
new file mode 100644
index 0000000..bcc9086
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/sparksql/CassandraDataSink.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.cassandra.spark.sparksql;
+
+import org.apache.cassandra.spark.bulkwriter.BulkWriterContext;
+import org.apache.cassandra.spark.bulkwriter.CassandraBulkSourceRelation;
+import org.apache.cassandra.spark.bulkwriter.CassandraBulkWriterContext;
+import org.apache.cassandra.spark.bulkwriter.JobInfo;
+import org.apache.cassandra.spark.bulkwriter.LoadNotSupportedException;
+import org.apache.cassandra.spark.utils.ScalaConversionUtils;
+import org.apache.spark.SparkContext;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SQLContext;
+import org.apache.spark.sql.SaveMode;
+import org.apache.spark.sql.sources.BaseRelation;
+import org.apache.spark.sql.sources.CreatableRelationProvider;
+import org.apache.spark.sql.sources.DataSourceRegister;
+import org.apache.spark.sql.types.StructType;
+import org.jetbrains.annotations.NotNull;
+import scala.collection.immutable.Map;
+
+public class CassandraDataSink implements DataSourceRegister, CreatableRelationProvider
+{
+    @Override
+    @NotNull
+    public String shortName()
+    {
+        return "cassandraBulkWrite";
+    }
+
+    /**
+     * @param sqlContext the SQLContext instance
+     * @param saveMode   must be {@link SaveMode#Append}
+     * @param parameters the writer options
+     * @param data       the data to persist into the Cassandra table
+     * @throws LoadNotSupportedException if the @<code>saveMode</code> is not supported: {@link SaveMode#Overwrite},
+     *                                   {@link SaveMode#ErrorIfExists}, or {@link SaveMode#Ignore}
+     */
+    @Override
+    @NotNull
+    public BaseRelation createRelation(@NotNull SQLContext sqlContext,
+                                       @NotNull SaveMode saveMode,
+                                       @NotNull Map<String, String> parameters,
+                                       @NotNull Dataset<Row> data)
+    {
+        switch (saveMode)
+        {
+            case Append:
+                // Initialize the job group ID for later use if we need to cancel the job
+                // TODO: Can we get a more descriptive "description" in here from the end user somehow?
+                BulkWriterContext writerContext = createBulkWriterContext(
+                        sqlContext.sparkContext(),
+                        ScalaConversionUtils.<String, String>mapAsJavaMap(parameters),
+                        data.schema());
+                try
+                {
+                    JobInfo jobInfo = writerContext.job();
+                    String description = "Cassandra Bulk Load for table " + jobInfo.getFullTableName();
+                    CassandraBulkSourceRelation relation = new CassandraBulkSourceRelation(writerContext, sqlContext);
+                    sqlContext.sparkContext().setJobGroup(jobInfo.getId().toString(), description, false);
+                    relation.insert(data, false);
+                    return relation;
+                }
+                catch (Exception exception)
+                {
+                    throw new RuntimeException(exception);
+                }
+                finally
+                {
+                    writerContext.shutdown();
+                    sqlContext.sparkContext().clearJobGroup();
+                }
+            case Overwrite:
+                throw new LoadNotSupportedException("SaveMode.Overwrite is not supported on Cassandra as it needs privileged TRUNCATE operation");
+            default:
+                throw new LoadNotSupportedException("SaveMode." + saveMode + " is not supported");
+        }
+    }
+
+    @NotNull
+    protected BulkWriterContext createBulkWriterContext(@NotNull SparkContext sparkContext,
+                                                        @NotNull java.util.Map<String, String> options,
+                                                        @NotNull StructType schema)
+    {
+        return CassandraBulkWriterContext.fromOptions(sparkContext, options, schema);
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/sparksql/FullRowBuilder.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/sparksql/FullRowBuilder.java
new file mode 100644
index 0000000..611f78b
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/sparksql/FullRowBuilder.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.cassandra.spark.sparksql;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.IntStream;
+
+import org.apache.cassandra.spark.data.CqlField;
+import org.apache.cassandra.spark.data.CqlTable;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+
+/**
+ * FullRowBuilder expects all fields in the schema to be returned, i.e. no prune column filter
+ */
+class FullRowBuilder implements RowBuilder
+{
+    static final Object[] EMPTY_RESULT = new Object[0];
+    final int numColumns;
+    final int numCells;
+    final boolean noValueColumns;
+    int extraColumns;
+    Object[] result;
+    int count;
+    private final CqlTable cqlTable;
+
+    FullRowBuilder(CqlTable cqlTable, boolean noValueColumns)
+    {
+        this.cqlTable = cqlTable;
+        this.numColumns = cqlTable.numFields();
+        this.numCells = cqlTable.numNonValueColumns() + (noValueColumns ? 0 : 1);
+        this.noValueColumns = noValueColumns;
+    }
+
+    @Override
+    public CqlTable getCqlTable()
+    {
+        return cqlTable;
+    }
+
+    @Override
+    public void reset()
+    {
+        this.count = 0;
+        int totalColumns = numColumns + extraColumns;
+        if (totalColumns != 0)
+        {
+            result = new Object[totalColumns];
+        }
+        else
+        {
+            result = EMPTY_RESULT;
+        }
+    }
+
+    @Override
+    public boolean isFirstCell()
+    {
+        return count == 0;
+    }
+
+    @Override
+    public void copyKeys(Cell cell)
+    {
+        // Need to handle special case where schema is only partition or clustering keys - i.e. no value columns
+        int length = noValueColumns || cell.isTombstone() ? cell.values.length : cell.values.length - 1;
+        System.arraycopy(cell.values, 0, result, 0, length);
+        count += length;
+    }
+
+    @Override
+    public void copyValue(Cell cell)
+    {
+        // Copy the next value column
+        result[cell.position] = cell.values[cell.values.length - 1];
+        count++;  // Increment the number of cells visited
+    }
+
+    @Override
+    public Object[] array()
+    {
+        return result;
+    }
+
+    @Override
+    public int columnsCount()
+    {
+        return numColumns;
+    }
+
+    @Override
+    public boolean hasRegularValueColumn()
+    {
+        return !noValueColumns;
+    }
+
+    @Override
+    public int expandRow(int extraColumns)
+    {
+        this.extraColumns = extraColumns;
+        return numColumns;
+    }
+
+    @Override
+    public boolean hasMoreCells()
+    {
+        return count < numColumns;
+    }
+
+    @Override
+    public void onCell(Cell cell)
+    {
+        assert 0 < cell.values.length && cell.values.length <= numCells;
+    }
+
+    @Override
+    public int fieldIndex(String name)
+    {
+        List<CqlField> fields = cqlTable.fields();
+        return IntStream.range(0, fields.size())
+                        .filter(i -> Objects.equals(fields.get(i).name(), name))
+                        .findFirst()
+                        .orElse(-1);
+    }
+
+    @Override
+    public GenericInternalRow build()
+    {
+        return new GenericInternalRow(result);
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/sparksql/SparkCellIterator.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/sparksql/SparkCellIterator.java
new file mode 100644
index 0000000..f40a88f
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/sparksql/SparkCellIterator.java
@@ -0,0 +1,412 @@
+/*
+ * 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.cassandra.spark.sparksql;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.spark.data.CqlField;
+import org.apache.cassandra.spark.data.CqlTable;
+import org.apache.cassandra.spark.data.DataLayer;
+import org.apache.cassandra.spark.reader.Rid;
+import org.apache.cassandra.spark.reader.StreamScanner;
+import org.apache.cassandra.spark.sparksql.filters.CdcOffsetFilter;
+import org.apache.cassandra.spark.sparksql.filters.PartitionKeyFilter;
+import org.apache.cassandra.spark.sparksql.filters.PruneColumnFilter;
+import org.apache.cassandra.spark.stats.Stats;
+import org.apache.cassandra.spark.utils.ArrayUtils;
+import org.apache.cassandra.spark.utils.ByteBufferUtils;
+import org.apache.cassandra.spark.utils.ColumnTypes;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Iterate through CompactionIterator, deserializing ByteBuffers and normalizing into Object[] array in column order
+ */
+public class SparkCellIterator implements Iterator<Cell>, AutoCloseable
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(SparkCellIterator.class);
+
+    protected final DataLayer dataLayer;
+    private final Stats stats;
+    private final CqlTable cqlTable;
+    private final Object[] values;
+    private final int numPartitionKeys;
+    private final boolean noValueColumns;
+    @Nullable
+    protected final PruneColumnFilter columnFilter;
+    private final long startTimeNanos;
+    @NotNull
+    private final StreamScanner scanner;
+    @NotNull
+    private final Rid rid;
+
+    // Mutable Iterator State
+    private boolean skipPartition = false;
+    private boolean newRow = false;
+    private boolean closed = false;
+    private Cell next = null;
+    private long previousTimeNanos;
+
+    protected final int partitionId;
+
+    public SparkCellIterator(int partitionId,
+                             @NotNull DataLayer dataLayer,
+                             @Nullable StructType requiredSchema,
+                             @NotNull List<PartitionKeyFilter> partitionKeyFilters,
+                             @Nullable CdcOffsetFilter cdcOffsetFilter)
+    {
+        this.partitionId = partitionId;
+        this.dataLayer = dataLayer;
+        stats = dataLayer.stats();
+        cqlTable = dataLayer.cqlTable();
+        numPartitionKeys = cqlTable.numPartitionKeys();
+        columnFilter = buildColumnFilter(requiredSchema, cqlTable);
+        if (columnFilter != null)
+        {
+            LOGGER.info("Adding prune column filter columns='{}'", String.join(",", columnFilter.requiredColumns()));
+
+            // If we are reading only partition/clustering keys or static columns, no value columns
+            Set<String> valueColumns = cqlTable.valueColumns().stream().map(CqlField::name).collect(Collectors.toSet());
+            noValueColumns = columnFilter.requiredColumns().stream().noneMatch(valueColumns::contains);
+        }
+        else
+        {
+            noValueColumns = cqlTable.numValueColumns() == 0;
+        }
+
+        // The value array copies across all the partition/clustering/static columns
+        // and the single column value for this cell to the SparkRowIterator
+        values = new Object[cqlTable.numNonValueColumns() + (noValueColumns ? 0 : 1)];
+
+        // Open compaction scanner
+        startTimeNanos = System.nanoTime();
+        previousTimeNanos = startTimeNanos;
+        scanner = openScanner(partitionId, partitionKeyFilters, cdcOffsetFilter);
+        long openTimeNanos = System.nanoTime() - startTimeNanos;
+        LOGGER.info("Opened CompactionScanner runtimeNanos={}", openTimeNanos);
+        stats.openedCompactionScanner(openTimeNanos);
+        rid = scanner.rid();
+        stats.openedSparkCellIterator();
+    }
+
+    protected StreamScanner openScanner(int partitionId,
+                                        @NotNull List<PartitionKeyFilter> partitionKeyFilters,
+                                        @Nullable CdcOffsetFilter cdcOffsetFilter)
+    {
+        return dataLayer.openCompactionScanner(partitionId, partitionKeyFilters, columnFilter);
+    }
+
+    @Nullable
+    static PruneColumnFilter buildColumnFilter(@Nullable StructType requiredSchema, @NotNull CqlTable cqlTable)
+    {
+        return requiredSchema != null
+                ? new PruneColumnFilter(Arrays.stream(requiredSchema.fields())
+                                              .map(StructField::name)
+                                              .filter(cqlTable::has)
+                                              .collect(Collectors.toSet()))
+                : null;
+    }
+
+    public boolean noValueColumns()
+    {
+        return noValueColumns;
+    }
+
+    @Override
+    public boolean hasNext()
+    {
+        try
+        {
+            return hasNextThrows();
+        }
+        catch (IOException exception)
+        {
+            throw new RuntimeException(exception);
+        }
+    }
+
+    public boolean hasNextThrows() throws IOException
+    {
+        if (next != null || closed)
+        {
+            return !closed;
+        }
+        return getNext();
+    }
+
+    @Override
+    public Cell next()
+    {
+        Cell result = next;
+        assert result != null;
+        next = null;
+        newRow = false;
+        long now = System.nanoTime();
+        stats.nextCell(now - previousTimeNanos);
+        previousTimeNanos = now;
+        return result;
+    }
+
+    private boolean getNext() throws IOException
+    {
+        while (scanner.hasNext())
+        {
+            // If hasNext returns true, it indicates the partition keys has been loaded into the rid.
+            // Therefore, let's try to rebuild partition.
+            // Deserialize partition keys - if we have moved to a new partition - and update 'values' Object[] array.
+            maybeRebuildPartition();
+
+            if (rid.shouldConsumeRangeTombstoneMarkers())
+            {
+                List<RangeTombstoneMarker> markers = rid.getRangeTombstoneMarkers();
+                long maxTimestamp = markers.stream()
+                                           .map(marker -> {
+                                               if (marker.isBoundary())
+                                               {
+                                                   return Math.max(marker.openDeletionTime(false), marker.closeDeletionTime(false));
+                                               }
+                                               else
+                                               {
+                                                   return marker.isOpen(false) ? marker.openDeletionTime(false) : marker.closeDeletionTime(false);
+                                               }
+                                           })
+                                           .max(Long::compareTo)
+                                           .get();  // Safe to call get as markers is non-empty
+                // Range tombstones requires only to have the partition key in the spark row,
+                // the range tombstones are encoded in the extra column
+                int partitionkeyLength = cqlTable.numPartitionKeys();
+                next = new RangeTombstone(ArrayUtils.retain(values, 0, partitionkeyLength), maxTimestamp, markers);
+                rid.resetRangeTombstoneMarkers();
+                return true;
+            }
+
+            if (rid.isPartitionDeletion())
+            {
+                // Special case that row deletion will only have the partition key parts present in the values array
+                int partitionkeyLength = cqlTable.numPartitionKeys();
+                // Strip out other values if any rather than the partition keys
+                next = new Tombstone(ArrayUtils.retain(values, 0, partitionkeyLength), rid.getTimestamp());
+                rid.setPartitionDeletion(false);  // Reset
+                return true;
+            }
+
+            scanner.advanceToNextColumn();
+
+            // Skip partition e.g. if token is outside of Spark worker token range
+            if (skipPartition)
+            {
+                continue;
+            }
+
+            // Deserialize clustering keys - if moved to new CQL row - and update 'values' Object[] array
+            ByteBuffer columnNameBuf = Objects.requireNonNull(rid.getColumnName(), "ColumnName buffer in Rid is null, this is unexpected");
+            maybeRebuildClusteringKeys(columnNameBuf);
+
+            // Deserialize CQL field column name
+            ByteBuffer component = ColumnTypes.extractComponent(columnNameBuf, cqlTable.numClusteringKeys());
+            String columnName = component != null ? ByteBufferUtils.stringThrowRuntime(component) : null;
+            if (columnName == null || columnName.length() == 0)
+            {
+                if (noValueColumns)
+                {
+                    // Special case where schema consists only of partition keys, clustering keys or static columns, no value columns
+                    next = new Cell(values, 0, newRow, rid.isUpdate(), rid.getTimestamp());
+                    return true;
+                }
+
+                // SBR job (not CDC) should not expect encountering a row tombstone.
+                // It would throw IllegalStateException at the beginning of this method (at scanner.hasNext()).
+                // For a row deletion, the resulting row tombstone does not carry other fields than the primary keys.
+                if (rid.isRowDeletion())
+                {
+                    // Special case that row deletion will only have the primary key parts present in the values array
+                    int primaryKeyLength = cqlTable.numPrimaryKeyColumns();
+                    // Strip out other values if any rather than the primary keys
+                    next = new Tombstone(ArrayUtils.retain(values, 0, primaryKeyLength), rid.getTimestamp());
+                    // Reset row deletion flag
+                    rid.setRowDeletion(false);
+                    return true;
+                }
+
+                continue;
+            }
+
+            CqlField field = cqlTable.getField(columnName);
+            if (field == null)
+            {
+                LOGGER.warn("Ignoring unknown column columnName='{}'", columnName);
+                continue;
+            }
+
+            // Deserialize value field or static column and update 'values' Object[] array
+            deserializeField(field);
+
+            // Static column, so continue reading entire CQL row before returning
+            if (field.isStaticColumn())
+            {
+                continue;
+            }
+
+            if (rid.hasCellTombstoneInComplex())
+            {
+                next = new TombstonesInComplex(values, field.position(), newRow, rid.getTimestamp(), columnName, rid.getCellTombstonesInComplex());
+                rid.resetCellTombstonesInComplex();
+            }
+            else
+            {
+                // Update next Cell
+                next = new Cell(values, field.position(), newRow, rid.isUpdate(), rid.getTimestamp());
+            }
+
+            return true;
+        }
+
+        // Finished so close
+        next = null;
+        close();
+        return false;
+    }
+
+    @Override
+    public void close() throws IOException
+    {
+        if (!closed)
+        {
+            scanner.close();
+            closed = true;
+            long runtimeNanos = System.nanoTime() - startTimeNanos;
+            LOGGER.info("Closed CompactionScanner runtimeNanos={}", runtimeNanos);
+            stats.closedSparkCellIterator(runtimeNanos);
+        }
+    }
+
+    /* Iterator Helpers */
+
+    /**
+     * If it is a new partition see if we can skip (e.g. if partition outside Spark worker token range), otherwise re-build partition keys
+     */
+    private void maybeRebuildPartition()
+    {
+        if (!rid.isNewPartition())
+        {
+            return;
+        }
+
+        // Skip partitions not in the token range for this Spark partition
+        newRow = true;
+
+        for (CqlField field : cqlTable.staticColumns())
+        {
+            // We need to reset static columns between partitions, if a static column is null/not-populated
+            // in the next partition, then the previous value might be carried across
+            values[field.position()] = null;
+        }
+
+        skipPartition = !dataLayer.isInPartition(partitionId, rid.getToken(), rid.getPartitionKey());
+        if (skipPartition)
+        {
+            stats.skippedPartitionInIterator(rid.getPartitionKey(), rid.getToken());
+            return;
+        }
+
+        // Or new partition, so deserialize partition keys and update 'values' array
+        ByteBuffer partitionKey = rid.getPartitionKey();
+        if (numPartitionKeys == 1)
+        {
+            // Not a composite partition key
+            CqlField field = cqlTable.partitionKeys().get(0);
+            values[field.position()] = deserialize(field, partitionKey);
+        }
+        else
+        {
+            // Split composite partition keys
+            ByteBuffer[] partitionKeyBufs = ColumnTypes.split(partitionKey, numPartitionKeys);
+            int index = 0;
+            for (CqlField field : cqlTable.partitionKeys())
+            {
+                values[field.position()] = deserialize(field, partitionKeyBufs[index++]);
+            }
+        }
+    }
+
+    /**
+     * Deserialize clustering key components and update 'values' array if changed. Mark isNewRow true if we move to new CQL row.
+     */
+    private void maybeRebuildClusteringKeys(@NotNull ByteBuffer columnNameBuf)
+    {
+        List<CqlField> clusteringKeys = cqlTable.clusteringKeys();
+        if (clusteringKeys.isEmpty())
+        {
+            return;
+        }
+
+        int index = 0;
+        for (CqlField field : clusteringKeys)
+        {
+            Object newObj = deserialize(field, ColumnTypes.extractComponent(columnNameBuf, index++));
+            Object oldObj = values[field.position()];
+            if (newRow || oldObj == null || newObj == null || !field.equals(newObj, oldObj))
+            {
+                newRow = true;
+                values[field.position()] = newObj;
+            }
+        }
+    }
+
+    /**
+     * Deserialize value field if required and update 'values' array
+     */
+    private void deserializeField(@NotNull CqlField field)
+    {
+        if (columnFilter == null || columnFilter.includeColumn(field.name()))
+        {
+            // Deserialize value
+            Object value = deserialize(field, rid.getValue());
+
+            if (field.isStaticColumn())
+            {
+                values[field.position()] = value;
+                return;
+            }
+
+            values[values.length - 1] = value;  // Last index in array always stores the cell value
+        }
+    }
+
+    private Object deserialize(CqlField field, ByteBuffer buffer)
+    {
+        long now = System.nanoTime();
+        Object value = buffer != null ? field.deserialize(buffer) : null;
+        stats.fieldDeserialization(field, System.nanoTime() - now);
+        return value;
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/utils/BuildInfo.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/utils/BuildInfo.java
new file mode 100644
index 0000000..c432258
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/utils/BuildInfo.java
@@ -0,0 +1,121 @@
+/*
+ * 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.cassandra.spark.utils;
+
+import java.io.InputStream;
+import java.util.Properties;
+import java.util.function.Function;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Utility for serving build related information
+ */
+public final class BuildInfo
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(BuildInfo.class);
+
+    // Do not reorder! Build version constants must be initialized before user agent constants.
+    public static final String APPLICATION_NAME = "cassandra-analytics";
+    public static final String BUILD_VERSION = getBuildVersion();
+    @SuppressWarnings("unused")  // Part of this library's API used by the consumers
+    public static final String BUILD_VERSION_AND_REVISION = getBuildVersionAndRevision();
+    public static final String READER_USER_AGENT = getUserAgent("reader");
+    public static final String WRITER_USER_AGENT = getUserAgent("writer");
+
+    private BuildInfo()
+    {
+        throw new IllegalStateException(getClass() + " is static utility class and shall not be instantiated");
+    }
+
+    private static String getUserAgent(String feature)
+    {
+        return String.format("%s/%s %s", APPLICATION_NAME, BUILD_VERSION, feature);
+    }
+
+    @VisibleForTesting
+    static String getBuildVersion()
+    {
+        return getBuildInfo(properties -> properties.getProperty("build-version"));
+    }
+
+    /**
+     * Determine whether the provided version is at least java 11
+     *
+     * @param version the java specification version
+     * @return {@code true} if the java version is at least java 11, {@code false} otherwise
+     */
+    public static boolean isAtLeastJava11(String version)
+    {
+        if (version == null)
+        {
+            return false;
+        }
+        else if (version.contains("."))
+        {
+            return version.compareTo("11") >= 0;
+        }
+        else
+        {
+            return Integer.parseInt(version) >= 11;
+        }
+    }
+
+    /**
+     * @return the java specification version from the system properties, or {@code null} if a
+     * {@link SecurityException} prevents us from reading the property
+     */
+    public static String javaSpecificationVersion()
+    {
+        String javaSpecificationVersionPropertyName = "java.specification.version";
+        try
+        {
+            return System.getProperty(javaSpecificationVersionPropertyName);
+        }
+        catch (SecurityException exception)
+        {
+            LOGGER.error("Unable to determine java specification version from system property={}",
+                         javaSpecificationVersionPropertyName, exception);
+            return null;
+        }
+    }
+
+    private static String getBuildVersionAndRevision()
+    {
+        return getBuildInfo(properties -> properties.getProperty("build-version") + "-" + properties.getProperty("build-rev"));
+    }
+
+    private static String getBuildInfo(Function<Properties, String> extractor)
+    {
+        try (InputStream in = BuildInfo.class.getResourceAsStream("/cassandra-analytics-build.properties"))
+        {
+            Properties properties = new Properties();
+            properties.load(in);
+            return extractor.apply(properties);
+        }
+        catch (Exception exception)
+        {
+            LOGGER.warn("Could not load the resource to get Cassandra Analytics build version", exception);
+            return "unknown";
+        }
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/utils/CqlUtils.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/utils/CqlUtils.java
new file mode 100644
index 0000000..5b14339
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/utils/CqlUtils.java
@@ -0,0 +1,291 @@
+/*
+ * 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.cassandra.spark.utils;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.cassandra.spark.data.ReplicationFactor;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * CQL-related utility methods
+ */
+public final class CqlUtils
+{
+    // Properties to be overridden when extracted from the table schema
+    private static final List<String> TABLE_PROPERTY_OVERRIDE_ALLOWLIST = Arrays.asList("min_index_interval",
+                                                                                        "max_index_interval",
+                                                                                        "cdc");
+    private static final Pattern REPLICATION_FACTOR_PATTERN = Pattern.compile("WITH REPLICATION = (\\{[^\\}]*\\})");
+    // Initialize a mapper allowing single quotes to process the RF string from the CREATE KEYSPACE statement
+    private static final ObjectMapper MAPPER = new ObjectMapper().configure(JsonParser.Feature.ALLOW_SINGLE_QUOTES, true);
+
+    private CqlUtils()
+    {
+        throw new IllegalStateException(getClass() + " is static utility class and shall not be instantiated");
+    }
+
+    public static String cleanCql(@NotNull String cql)
+    {
+        return cql.replaceAll("(\\\\r|\\\\n|\\\\r\\n)+", "\n")
+                  .replaceAll("\n", "")
+                  .replaceAll("\\\\", "");
+    }
+
+    private static String removeTableProps(@NotNull String schema)
+    {
+        int index = schema.indexOf('(');
+        int count = 1;
+        if (index < 0)
+        {
+            throw new RuntimeException("Missing parentheses in table schema " + schema);
+        }
+        while (++index < schema.length())  // Find closing bracket
+        {
+            if (schema.charAt(index) == ')')
+            {
+                count--;
+            }
+            else if (schema.charAt(index) == '(')
+            {
+                count++;
+            }
+            if (count == 0)
+            {
+                break;
+            }
+        }
+        if (count > 0)
+        {
+            throw new RuntimeException("Found unbalanced parentheses in table schema " + schema);
+        }
+        return schema.substring(0, index + 1);
+    }
+
+    /**
+     * The schema might contain quotes, but on-disk the table path doesn't have quotes, so causes problems for create/list snapshot
+     *
+     * @param table table name
+     * @return return a cleaned table table without quotes
+     */
+    public static String cleanTableName(@Nullable String table)
+    {
+        if (table == null)
+        {
+            return null;
+        }
+        while (table.startsWith("\"") && table.endsWith("\""))
+        {
+            table = table.substring(1, table.length() - 1);
+        }
+        return table;
+    }
+
+    /**
+     * @param schemaStr full cluster schema text
+     * @return map of keyspace/table identifier to table create statements
+     */
+    public static Map<TableIdentifier, String> extractCdcTables(@NotNull String schemaStr)
+    {
+        String cleaned = cleanCql(schemaStr);
+        Pattern pattern = Pattern.compile("CREATE TABLE \"?(\\w+)\"?\\.\"?(\\w+)\"?[^;]*cdc = true[^;]*;");
+        Matcher matcher = pattern.matcher(cleaned);
+        Map<TableIdentifier, String> createStmts = new HashMap<>();
+        while (matcher.find())
+        {
+            String keyspace = matcher.group(1);
+            String table = matcher.group(2);
+            createStmts.put(TableIdentifier.of(keyspace, table), extractCleanedTableSchema(cleaned, keyspace, table));
+        }
+        return createStmts;
+    }
+
+    public static Set<String> extractKeyspaceNames(@NotNull String schemaStr)
+    {
+        String cleaned = cleanCql(schemaStr);
+        Pattern pattern = Pattern.compile("CREATE KEYSPACE \"?(\\w+)?\"? [^;]*;");
+        Matcher matcher = pattern.matcher(cleaned);
+
+        Set<String> keyspaces = new HashSet<>();
+        while (matcher.find())
+        {
+            String keyspace = matcher.group(1);
+            if (!keyspace.startsWith("system_") && !keyspace.startsWith("cie_"))
+            {
+                keyspaces.add(keyspace);
+            }
+        }
+        return keyspaces;
+    }
+
+    public static String extractKeyspaceSchema(@NotNull String schemaStr, @NotNull String keyspace)
+    {
+        String cleaned = cleanCql(schemaStr);
+        Pattern pattern = Pattern.compile(String.format("CREATE KEYSPACE \"?%s?\"? [^;]*;", keyspace));
+        Matcher matcher = pattern.matcher(cleaned);
+
+        if (!matcher.find())
+        {
+            throw new RuntimeException(String.format("Could not find schema for keyspace: %s", keyspace));
+        }
+
+        return cleaned.substring(matcher.start(0), matcher.end(0));
+    }
+
+    public static ReplicationFactor extractReplicationFactor(@NotNull String schemaStr, @NotNull String keyspace)
+    {
+        String createKeyspaceSchema = extractKeyspaceSchema(schemaStr, keyspace);
+        Matcher matcher = REPLICATION_FACTOR_PATTERN.matcher(createKeyspaceSchema);
+
+        if (!matcher.find())
+        {
+            throw new RuntimeException(String.format("Could not find replication factor for keyspace: %s", keyspace));
+        }
+
+        Map<String, String> map;
+        try
+        {
+            map = MAPPER.readValue(matcher.group(1), new TypeReference<Map<String, String>>() {});  // CHECKSTYLE IGNORE: Empty anonymous inner class
+        }
+        catch (IOException exception)
+        {
+            throw new RuntimeException(String.format("Unable to parse replication factor for keyspace: %s", keyspace), exception);
+        }
+
+        String className = map.remove("class");
+        ReplicationFactor.ReplicationStrategy strategy = ReplicationFactor.ReplicationStrategy.getEnum(className);
+        return new ReplicationFactor(strategy, map.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, v -> Integer.parseInt(v.getValue()))));
+    }
+
+    public static String extractTableSchema(@NotNull String schemaStr, @NotNull String keyspace, @NotNull String table)
+    {
+        return extractCleanedTableSchema(cleanCql(schemaStr), keyspace, table);
+    }
+
+    public static String extractCleanedTableSchema(@NotNull String cleaned,
+                                                   @NotNull String keyspace,
+                                                   @NotNull String table)
+    {
+        Pattern pattern = Pattern.compile(String.format("CREATE TABLE ?\"?%s?\"?\\.{1}\"?%s\"?[^;]*;", keyspace, table));
+        Matcher matcher = pattern.matcher(cleaned);
+        if (matcher.find())
+        {
+            String fullSchema = cleaned.substring(matcher.start(0), matcher.end(0));
+            String tableOnly = removeTableProps(fullSchema);
+            String quotedTableName = String.format("\"%s\"", table);
+            if (tableOnly.contains(quotedTableName))
+            {
+                // Remove quoted table name from schema
+                tableOnly = tableOnly.replaceFirst(quotedTableName, table);
+            }
+            String redactedSchema = tableOnly;
+            String clustering = extractClustering(fullSchema);
+            String separator = " WITH ";
+            if (clustering != null)
+            {
+                redactedSchema = redactedSchema + separator + clustering;
+                separator = " AND ";
+            }
+
+            List<String> propStrings = extractOverrideProperties(fullSchema, TABLE_PROPERTY_OVERRIDE_ALLOWLIST);
+            if (!propStrings.isEmpty())
+            {
+                redactedSchema = redactedSchema + separator + String.join(" AND ", propStrings);
+                separator = " AND ";  // For completeness
+            }
+
+            return redactedSchema + ";";
+        }
+        throw new RuntimeException(String.format("Could not find schema for table: %s.%s", keyspace, table));
+    }
+
+    @VisibleForTesting
+    static List<String> extractOverrideProperties(String schemaStr, List<String> properties)
+    {
+
+        List<String> overrideTableProps = new ArrayList<>();
+        if (properties.isEmpty())
+        {
+            return overrideTableProps;
+        }
+        Pattern pattern = Pattern.compile("(" + properties.stream().collect(Collectors.joining("|")) + ") = (\\w+)");
+        Matcher matcher = pattern.matcher(schemaStr);
+
+        while (matcher.find())
+        {
+            String parsedProp = schemaStr.substring(matcher.start(), matcher.end());
+            overrideTableProps.add(parsedProp);
+        }
+        return overrideTableProps;
+    }
+
+    @VisibleForTesting
+    static String extractClustering(String schemaStr)
+    {
+        Pattern pattern = Pattern.compile("CLUSTERING ORDER BY \\([^)]*");
+        Matcher matcher = pattern.matcher(schemaStr);
+        if (matcher.find())
+        {
+            return schemaStr.substring(matcher.start(0), matcher.end(0) + 1);
+        }
+        return null;
+    }
+
+    public static Set<String> extractUdts(@NotNull String schemaStr, @NotNull String keyspace)
+    {
+        Pattern pattern = Pattern.compile(String.format("CREATE TYPE \"?%s\"?\\.{1}[^;]*;", keyspace));
+        Matcher matcher = pattern.matcher(schemaStr);
+        Set<String> result = new HashSet<>();
+        while (matcher.find())
+        {
+            result.add(cleanCql(matcher.group()));
+        }
+        return result;
+    }
+
+    public static int extractIndexCount(@NotNull String schemaStr, @NotNull String keyspace, @NotNull String table)
+    {
+        String cleaned = cleanCql(schemaStr);
+        Pattern pattern = Pattern.compile(String.format("CREATE (CUSTOM )?INDEX \"?[^ ]* ON ?\"?%s?\"?\\.{1}\"?%s\"?[^;]*;", keyspace, table));
+        Matcher matcher = pattern.matcher(cleaned);
+        int indexCount = 0;
+        while (matcher.find())
+        {
+            indexCount++;
+        }
+        return indexCount;
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/utils/FilterUtils.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/utils/FilterUtils.java
new file mode 100644
index 0000000..05e4812
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/utils/FilterUtils.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.cassandra.spark.utils;
+
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.spark.sql.sources.EqualTo;
+import org.apache.spark.sql.sources.Filter;
+import org.apache.spark.sql.sources.In;
+
+public final class FilterUtils
+{
+    private FilterUtils()
+    {
+        throw new IllegalStateException(getClass() + " is static utility class and shall not be instantiated");
+    }
+
+    public static Map<String, List<String>> extractPartitionKeyValues(Filter[] filters,
+                                                                      Set<String> partitionKeyColumnNames)
+    {
+        HashMap<String, List<String>> keyValuePairs = new HashMap<>();
+
+        Arrays.stream(filters)
+                .filter(filter -> filter instanceof EqualTo || filter instanceof In)
+                .forEach(filter -> {
+                    final AbstractMap.SimpleEntry<String, List<String>> filterKeyValue = filter instanceof EqualTo
+                            ? new AbstractMap.SimpleEntry<>(((EqualTo) filter).attribute(),
+                                                            Collections.singletonList(((EqualTo) filter).value().toString()))
+                            : new AbstractMap.SimpleEntry<>(((In) filter).attribute(),
+                                                            Arrays.stream(((In) filter).values())
+                                                                  .map(Object::toString)
+                                                                  .collect(Collectors.toList()));
+
+                    Preconditions.checkArgument(!keyValuePairs.containsKey(filterKeyValue.getKey()),
+                                                "Duplicate filter passed for key " + filterKeyValue.getKey());
+                    if (partitionKeyColumnNames.contains(filterKeyValue.getKey()))
+                    {
+                        keyValuePairs.put(filterKeyValue.getKey(), filterKeyValue.getValue());
+                    }
+                });
+
+        if (keyValuePairs.size() == 0)
+        {
+            return Collections.emptyMap();
+        }
+
+        Preconditions.checkArgument(keyValuePairs.keySet().containsAll(partitionKeyColumnNames),
+                                    "Invalid filter, all partition key parts must be restricted by = or in");
+        return keyValuePairs;
+    }
+
+    public static List<List<String>> cartesianProduct(List<List<String>> orderedValues)
+    {
+        List<List<String>> combinations = new ArrayList<>();
+
+        Preconditions.checkArgument(orderedValues.stream().noneMatch(List::isEmpty));
+        int sizeOfProduct = orderedValues.size();
+        int[] indices = new int[sizeOfProduct];
+
+        while (true)
+        {
+            List<String> currProduct = new ArrayList<>();
+            for (int index = 0; index < sizeOfProduct; index++)
+            {
+                currProduct.add(orderedValues.get(index).get(indices[index]));
+            }
+            combinations.add(currProduct);
+
+            int position = 0;
+            while (position < sizeOfProduct && indices[position] + 1 >= orderedValues.get(position).size())
+            {
+                position++;
+            }
+
+            if (position == sizeOfProduct)
+            {
+                return combinations;
+            }
+
+            indices[position]++;
+            for (int index = position - 1; index >= 0; index--)
+            {
+                indices[index] = 0;
+            }
+        }
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/utils/Properties.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/utils/Properties.java
new file mode 100644
index 0000000..06c0056
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/utils/Properties.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.cassandra.spark.utils;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.spark.data.FileType;
+
+public final class Properties
+{
+    private static final long KIBI_BYTES = 1024;
+    private static final long MEBI_BYTES = 1024 * KIBI_BYTES;
+
+    // Sidecar
+    public static final int DEFAULT_SIDECAR_PORT = 9043;
+    public static final long DEFAULT_MAX_MILLIS_TO_SLEEP = TimeUnit.SECONDS.toMillis(60);
+    public static final int DEFAULT_MAX_RETRIES = 10;
+    public static final long DEFAULT_MILLIS_TO_SLEEP = 500;
+    public static final int DEFAULT_MAX_POOL_SIZE = 64;
+    public static final long DEFAULT_MAX_BUFFER_SIZE = 6 * MEBI_BYTES;
+    public static final long DEFAULT_CHUNK_BUFFER_SIZE = 4 * MEBI_BYTES;
+    public static final Map<FileType, Long> DEFAULT_MAX_BUFFER_OVERRIDE = ImmutableMap.of(
+            FileType.INDEX,            128 * KIBI_BYTES,
+            FileType.SUMMARY,          256 * KIBI_BYTES,
+            FileType.STATISTICS,       128 * KIBI_BYTES,
+            FileType.COMPRESSION_INFO, 128 * KIBI_BYTES,
+            FileType.COMMITLOG,         64 * MEBI_BYTES);
+    public static final Map<FileType, Long> DEFAULT_CHUNK_BUFFER_OVERRIDE = ImmutableMap.of(
+            FileType.INDEX,             32 * KIBI_BYTES,
+            FileType.SUMMARY,          128 * KIBI_BYTES,
+            FileType.STATISTICS,        64 * KIBI_BYTES,
+            FileType.COMPRESSION_INFO,  64 * KIBI_BYTES,
+            FileType.COMMITLOG,         64 * MEBI_BYTES);
+    public static final int DEFAULT_TIMEOUT_SECONDS = (int) TimeUnit.MINUTES.toSeconds(10);
+
+    // Expansion and Shrink
+    public static final Set<String> NODE_STATUS_NOT_CONSIDERED = new HashSet<>(Arrays.asList("JOINING", "DOWN"));
+
+    private Properties()
+    {
+        throw new IllegalStateException(getClass() + " is static utility class and shall not be instantiated");
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/utils/RangeUtils.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/utils/RangeUtils.java
new file mode 100644
index 0000000..86e391a
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/utils/RangeUtils.java
@@ -0,0 +1,168 @@
+/*
+ * 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.cassandra.spark.utils;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.BoundType;
+import com.google.common.collect.Multimap;
+import com.google.common.collect.Range;
+
+import org.apache.cassandra.bridge.TokenRange;
+import org.apache.cassandra.spark.data.partitioner.CassandraInstance;
+import org.apache.cassandra.spark.data.partitioner.Partitioner;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Common Cassandra range operations on Guava ranges. Assumes ranges are not wrapped around.
+ * It's the responsibility of caller to unwrap ranges. For example, [100, 1] should become
+ * [100, MAX] and [MIN, 1]. MIN and MAX depend on {@link Partitioner}.
+ */
+public final class RangeUtils
+{
+    private RangeUtils()
+    {
+        throw new IllegalStateException(getClass() + " is static utility class and shall not be instantiated");
+    }
+
+    @NotNull
+    private static BigInteger sizeOf(@NotNull Range<BigInteger> range)
+    {
+        Preconditions.checkArgument(range.lowerEndpoint().compareTo(range.upperEndpoint()) <= 0,
+                                    "RangeUtils assume ranges are not wrap-around");
+
+        if (range.isEmpty())
+        {
+            return BigInteger.ZERO;
+        }
+
+        BigInteger size = range.upperEndpoint().subtract(range.lowerEndpoint()).add(BigInteger.ONE);
+
+        if (range.lowerBoundType() == BoundType.OPEN)
+        {
+            size = size.subtract(BigInteger.ONE);
+        }
+
+        if (range.upperBoundType() == BoundType.OPEN)
+        {
+            size = size.subtract(BigInteger.ONE);
+        }
+
+        return size;
+    }
+
+    /**
+     * Splits the given range into equal sized small ranges. Number of splits can be controlled by
+     * numberOfSplits. If numberOfSplits are smaller than size of the range, split size would be set to 1.
+     *
+     * This is best effort scheme, numberOfSplits not necessarily as promised and not all splits may not be
+     * exact same size.
+     *
+     * @param range          the range to split
+     * @param numberOfSplits the desired number of splits
+     * @return the list of split ranges
+     */
+    @NotNull
+    public static List<Range<BigInteger>> split(@NotNull Range<BigInteger> range, int numberOfSplits)
+    {
+        Preconditions.checkArgument(range.lowerEndpoint().compareTo(range.upperEndpoint()) <= 0,
+                                    "RangeUtils assume ranges are not wrap-around");
+
+        if (range.isEmpty())
+        {
+            return Collections.emptyList();
+        }
+
+        // Make sure split size is not 0
+        BigInteger splitSize = sizeOf(range).divide(BigInteger.valueOf(numberOfSplits)).max(BigInteger.ONE);
+
+        // Start from range lower endpoint and spit ranges of size splitSize, until we cross the range
+        BigInteger nextLowerEndpoint = range.lowerBoundType() == BoundType.CLOSED ? range.lowerEndpoint() : range.lowerEndpoint().add(BigInteger.ONE);
+        List<Range<BigInteger>> splits = new ArrayList<>();
+        while (range.contains(nextLowerEndpoint))
+        {
+            BigInteger upperEndpoint = nextLowerEndpoint.add(splitSize);
+            splits.add(range.intersection(Range.closedOpen(nextLowerEndpoint, upperEndpoint)));
+            nextLowerEndpoint = upperEndpoint;
+        }
+
+        return splits;
+    }
+
+    @NotNull
+    public static Multimap<CassandraInstance, Range<BigInteger>> calculateTokenRanges(
+            @NotNull List<CassandraInstance> instances,
+            int replicationFactor,
+            @NotNull Partitioner partitioner)
+    {
+        Preconditions.checkArgument(replicationFactor != 0, "RF cannot be 0");
+        Preconditions.checkArgument(instances.size() == 0 || replicationFactor <= instances.size(),
+                String.format("RF (%d) cannot be greater than the number of Cassandra instances (%d)",
+                              replicationFactor, instances.size()));
+        Multimap<CassandraInstance, Range<BigInteger>> tokenRanges = ArrayListMultimap.create();
+        for (int index = 0; index < instances.size(); index++)
+        {
+            CassandraInstance instance = instances.get(index);
+            int disjointReplica = ((instances.size() + index) - replicationFactor) % instances.size();
+            BigInteger rangeStart = new BigInteger(instances.get(disjointReplica).token());
+            BigInteger rangeEnd = new BigInteger(instance.token());
+
+            // If start token is not strictly smaller than end token we are looking at a wrap around range, split it
+            if (rangeStart.compareTo(rangeEnd) >= 0)
+            {
+                tokenRanges.put(instance, Range.range(rangeStart, BoundType.OPEN, partitioner.maxToken(), BoundType.CLOSED));
+                tokenRanges.put(instance, Range.range(partitioner.minToken(), BoundType.CLOSED, rangeEnd, BoundType.CLOSED));
+            }
+            else
+            {
+                tokenRanges.put(instance, Range.openClosed(rangeStart, rangeEnd));
+            }
+        }
+
+        return tokenRanges;
+    }
+
+    @NotNull
+    public static TokenRange toTokenRange(@NotNull Range<BigInteger> range)
+    {
+        BigInteger lowerEndpoint = range.lowerEndpoint();
+        if (range.lowerBoundType() == BoundType.OPEN)
+        {
+            lowerEndpoint = lowerEndpoint.add(BigInteger.ONE);
+        }
+        BigInteger upperEndpoint = range.upperEndpoint();
+        if (range.upperBoundType() == BoundType.OPEN)
+        {
+            upperEndpoint = upperEndpoint.subtract(BigInteger.ONE);
+        }
+        return TokenRange.closed(lowerEndpoint, upperEndpoint);
+    }
+
+    @NotNull
+    public static Range<BigInteger> fromTokenRange(@NotNull TokenRange range)
+    {
+        return Range.closed(range.lowerEndpoint(), range.upperEndpoint());
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/utils/ScalaFunctions.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/utils/ScalaFunctions.java
new file mode 100644
index 0000000..3226156
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/utils/ScalaFunctions.java
@@ -0,0 +1,53 @@
+/*
+ * 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.cassandra.spark.utils;
+
+import scala.Function0;
+import scala.runtime.AbstractFunction0;
+import scala.runtime.BoxedUnit;
+
+public final class ScalaFunctions
+{
+    private ScalaFunctions()
+    {
+        throw new IllegalStateException(getClass() + " is static utility class and shall not be instantiated");
+    }
+
+    public static Function0<BoxedUnit> wrapLambda(Runnable funcToWrap)
+    {
+        return new Function0Wrapper(funcToWrap);
+    }
+
+    public static class Function0Wrapper extends AbstractFunction0<BoxedUnit>
+    {
+        private final Runnable funcToWrap;
+
+        public Function0Wrapper(Runnable funcToWrap)
+        {
+            this.funcToWrap = funcToWrap;
+        }
+
+        public BoxedUnit apply()
+        {
+            funcToWrap.run();
+            return BoxedUnit.UNIT;
+        }
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/utils/TableIdentifier.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/utils/TableIdentifier.java
new file mode 100644
index 0000000..ec936a6
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/utils/TableIdentifier.java
@@ -0,0 +1,81 @@
+/*
+ * 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.cassandra.spark.utils;
+
+import java.util.Objects;
+
+/**
+ * Qualified table name in Cassandra
+ */
+public class TableIdentifier
+{
+    private final String keyspace;
+    private final String table;
+
+    public TableIdentifier(String keyspace, String table)
+    {
+        this.keyspace = keyspace;
+        this.table = table;
+    }
+
+    public static TableIdentifier of(String keyspace, String table)
+    {
+        return new TableIdentifier(keyspace, table);
+    }
+
+    public String keyspace()
+    {
+        return keyspace;
+    }
+
+    public String table()
+    {
+        return table;
+    }
+
+    @Override
+    public boolean equals(Object other)
+    {
+        if (this == other)
+        {
+            return true;
+        }
+        if (other == null || this.getClass() != other.getClass())
+        {
+            return false;
+        }
+
+        TableIdentifier that = (TableIdentifier) other;
+        return Objects.equals(this.keyspace, that.keyspace)
+            && Objects.equals(this.table, that.table);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Objects.hash(keyspace, table);
+    }
+
+    @Override
+    public String toString()
+    {
+        return String.format("TableIdentifier{keyspace='%s', table='%s'}", keyspace, table);
+    }
+}
diff --git a/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/utils/UUIDs.java b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/utils/UUIDs.java
new file mode 100644
index 0000000..0d90472
--- /dev/null
+++ b/cassandra-analytics-core/src/main/java/org/apache/cassandra/spark/utils/UUIDs.java
@@ -0,0 +1,90 @@
+/*
+ * 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.cassandra.spark.utils;
+
+import java.security.SecureRandom;
+import java.util.Calendar;
+import java.util.TimeZone;
+import java.util.UUID;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * This code is borrowed from the Cassandra code in {@code org.apache.cassandra.utils.TimeUUID} with slight
+ * modifications. The code was copied over to remove the dependency we had on the cassandra client library.
+ */
+public final class UUIDs
+{
+    private static final long START_EPOCH = makeEpoch();
+    private static final long CLOCK_SEQ_AND_NODE = makeClockSeqAndNode();
+
+    private UUIDs()
+    {
+        throw new IllegalStateException(getClass() + " is static utility class and shall not be instantiated");
+    }
+
+    public static UUID startOf(long timestamp)
+    {
+        return new UUID(makeMSB(fromUnixTimestamp(timestamp)), CLOCK_SEQ_AND_NODE);
+    }
+
+    @VisibleForTesting
+    static long fromUnixTimestamp(long timestamp)
+    {
+        return (timestamp - START_EPOCH) * 10000;
+    }
+
+    @VisibleForTesting
+    static long makeMSB(long timestamp)
+    {
+        long msb = 0L;
+        msb |= (0x00000000FFFFFFFFL & timestamp)  << 32;
+        msb |= (0x0000FFFF00000000L & timestamp) >>> 16;
+        msb |= (0x0FFF000000000000L & timestamp) >>> 48;
+        msb |=  0x0000000000001000L;  // Sets the version to 1
+        return msb;
+    }
+
+    private static long makeEpoch()
+    {
+        // UUID v1 timestamp must be in 100-nanoseconds interval since 00:00:00.000 15 Oct 1582
+        Calendar calendar = Calendar.getInstance(TimeZone.getTimeZone("GMT-0"));
+        calendar.set(Calendar.YEAR, 1582);
+        calendar.set(Calendar.MONTH, Calendar.OCTOBER);
+        calendar.set(Calendar.DAY_OF_MONTH, 15);
+        calendar.set(Calendar.HOUR_OF_DAY, 0);
+        calendar.set(Calendar.MINUTE, 0);
+        calendar.set(Calendar.SECOND, 0);
+        calendar.set(Calendar.MILLISECOND, 0);
+        return calendar.getTimeInMillis();
+    }
+
+    private static long makeClockSeqAndNode()
+    {
+        SecureRandom secureRandom = new SecureRandom();
+        long clock = secureRandom.nextLong();
+
+        long lsb = 0;
+        lsb |= 0x8000000000000000L;                  // Variant (2 bits)
+        lsb |= (clock & 0x0000000000003FFFL) << 48;  // Clock sequence (14 bits)
+        lsb |= secureRandom.nextLong();              // 6 bytes
+        return lsb;
+    }
+}
diff --git a/cassandra-analytics-core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/cassandra-analytics-core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
new file mode 100644
index 0000000..9e699ad
--- /dev/null
+++ b/cassandra-analytics-core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
@@ -0,0 +1,2 @@
+org.apache.cassandra.spark.sparksql.CassandraDataSource
+org.apache.cassandra.spark.sparksql.CassandraDataSink
diff --git a/cassandra-analytics-core/src/main/spark2/org/apache/cassandra/spark/sparksql/CassandraDataSource.java b/cassandra-analytics-core/src/main/spark2/org/apache/cassandra/spark/sparksql/CassandraDataSource.java
new file mode 100644
index 0000000..68a7cfa
--- /dev/null
+++ b/cassandra-analytics-core/src/main/spark2/org/apache/cassandra/spark/sparksql/CassandraDataSource.java
@@ -0,0 +1,220 @@
+/*
+ * 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.cassandra.spark.sparksql;
+
+import java.io.Serializable;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.bridge.CassandraBridgeFactory;
+import org.apache.cassandra.spark.data.CassandraDataLayer;
+import org.apache.cassandra.spark.data.CassandraDataSourceHelper;
+import org.apache.cassandra.spark.data.CqlField;
+import org.apache.cassandra.spark.data.DataLayer;
+import org.apache.cassandra.spark.sparksql.filters.PartitionKeyFilter;
+import org.apache.cassandra.spark.utils.FilterUtils;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.sources.DataSourceRegister;
+import org.apache.spark.sql.sources.Filter;
+import org.apache.spark.sql.sources.v2.DataSourceOptions;
+import org.apache.spark.sql.sources.v2.DataSourceV2;
+import org.apache.spark.sql.sources.v2.ReadSupport;
+import org.apache.spark.sql.sources.v2.reader.DataSourceReader;
+import org.apache.spark.sql.sources.v2.reader.InputPartition;
+import org.apache.spark.sql.sources.v2.reader.InputPartitionReader;
+import org.apache.spark.sql.sources.v2.reader.SupportsPushDownFilters;
+import org.apache.spark.sql.sources.v2.reader.SupportsPushDownRequiredColumns;
+import org.apache.spark.sql.sources.v2.reader.partitioning.Distribution;
+import org.apache.spark.sql.sources.v2.reader.partitioning.Partitioning;
+import org.apache.spark.sql.types.StructType;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * A concrete implementation for the {@link CassandraDataSource}
+ */
+public class CassandraDataSource implements DataSourceV2, ReadSupport, DataSourceRegister
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(CassandraDataSource.class);
+    private DataLayer dataLayer;
+
+    public CassandraDataSource()
+    {
+        CassandraBridgeFactory.validateBridges();
+    }
+
+    @Override
+    public String shortName()
+    {
+        return "cassandraBulkRead";
+    }
+
+    @Override
+    public DataSourceReader createReader(DataSourceOptions options)
+    {
+        if (dataLayer == null)
+        {
+            dataLayer = getDataLayer(options);
+        }
+        return new SSTableSourceReader(dataLayer);
+    }
+
+    public DataLayer getDataLayer(DataSourceOptions options)
+    {
+        return CassandraDataSourceHelper.getDataLayer(options.asMap(), this::initializeDataLayer);
+    }
+
+    @VisibleForTesting
+    void initializeDataLayer(CassandraDataLayer dataLayer, CassandraDataLayer.ClientConfig config)
+    {
+        dataLayer.initialize(config);
+    }
+
+    public static class SSTableSourceReader
+            implements DataSourceReader, Serializable, SupportsPushDownFilters, SupportsPushDownRequiredColumns, Partitioning
+    {
+        private static final long serialVersionUID = -6622216100571485739L;
+        private Filter[] pushedFilters = new Filter[0];
+        private final DataLayer dataLayer;
+        private StructType requiredSchema = null;
+
+        SSTableSourceReader(@NotNull DataLayer dataLayer)
+        {
+            this.dataLayer = dataLayer;
+        }
+
+        @Override
+        public StructType readSchema()
+        {
+            return dataLayer.structType();
+        }
+
+        @Override
+        public List<InputPartition<InternalRow>> planInputPartitions()
+        {
+            List<PartitionKeyFilter> partitionKeyFilters = new ArrayList<>();
+
+            List<String> partitionKeyColumnNames = dataLayer.cqlTable().partitionKeys().stream()
+                                                                                       .map(CqlField::name)
+                                                                                       .collect(Collectors.toList());
+            Map<String, List<String>> partitionKeyValues =
+                    FilterUtils.extractPartitionKeyValues(pushedFilters, new HashSet<>(partitionKeyColumnNames));
+            if (partitionKeyValues.size() > 0)
+            {
+                List<List<String>> orderedValues = partitionKeyColumnNames.stream()
+                                                                          .map(partitionKeyValues::get)
+                                                                          .collect(Collectors.toList());
+                FilterUtils.cartesianProduct(orderedValues).forEach(keys -> {
+                    AbstractMap.SimpleEntry<ByteBuffer, BigInteger> filterKey =
+                            dataLayer.bridge().getPartitionKey(dataLayer.cqlTable(), dataLayer.partitioner(), keys);
+                    partitionKeyFilters.add(PartitionKeyFilter.create(filterKey.getKey(), filterKey.getValue()));
+                });
+            }
+            LOGGER.info("Creating data reader factories numPartitions={}", dataLayer.partitionCount());
+            return IntStream.range(0, dataLayer.partitionCount())
+                            .mapToObj(partitionId -> new SerializableInputPartition(partitionId, dataLayer, requiredSchema, partitionKeyFilters))
+                            .collect(Collectors.toList());
+        }
+
+        public static class SerializableInputPartition implements InputPartition<InternalRow>
+        {
+            private static final long serialVersionUID = -7916492108742137769L;
+            private final int partitionId;
+            @NotNull
+            private final DataLayer dataLayer;
+            @NotNull
+            private final StructType requiredSchema;
+            @NotNull
+            private final List<PartitionKeyFilter> partitionKeyFilters;
+
+            public SerializableInputPartition(int partitionId,
+                                              @NotNull DataLayer dataLayer,
+                                              @NotNull StructType requiredSchema,
+                                              @NotNull List<PartitionKeyFilter> partitionKeyFilters)
+            {
+                this.partitionId = partitionId;
+                this.dataLayer = dataLayer;
+                this.requiredSchema = requiredSchema;
+                this.partitionKeyFilters = partitionKeyFilters;
+            }
+
+            @Override
+            @NotNull
+            public InputPartitionReader<InternalRow> createPartitionReader()
+            {
+                return new SparkRowIterator(partitionId, dataLayer, requiredSchema, partitionKeyFilters);
+            }
+        }
+
+        /**
+         * Pushes down filters, and returns filters that need to be evaluated after scanning
+         *
+         * @param filters the filters in the query
+         * @return filters that need to be evaluated after scanning
+         */
+        @Override
+        public Filter[] pushFilters(Filter[] filters)
+        {
+            Filter[] unsupportedFilters = dataLayer.unsupportedPushDownFilters(filters);
+
+            List<Filter> supportedFilters = Lists.newArrayList(filters);
+            supportedFilters.removeAll(Arrays.asList(unsupportedFilters));
+            pushedFilters = supportedFilters.stream().toArray(Filter[]::new);
+
+            return unsupportedFilters;
+        }
+
+        @Override
+        public Filter[] pushedFilters()
+        {
+            return pushedFilters;
+        }
+
+        @Override
+        public void pruneColumns(StructType requiredSchema)
+        {
+            this.requiredSchema = requiredSchema;
+        }
+
+        @Override
+        public int numPartitions()
+        {
+            return dataLayer.partitionCount();
+        }
+
+        @Override
+        public boolean satisfy(Distribution distribution)
+        {
+            return true;
+        }
+    }
+}
diff --git a/cassandra-analytics-core/src/main/spark2/org/apache/cassandra/spark/sparksql/LocalDataSource.java b/cassandra-analytics-core/src/main/spark2/org/apache/cassandra/spark/sparksql/LocalDataSource.java
new file mode 100644
index 0000000..135ffe5
--- /dev/null
+++ b/cassandra-analytics-core/src/main/spark2/org/apache/cassandra/spark/sparksql/LocalDataSource.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.cassandra.spark.sparksql;
+
+import org.apache.cassandra.spark.data.DataLayer;
+import org.apache.cassandra.spark.data.LocalDataLayer;
+import org.apache.spark.sql.sources.v2.DataSourceOptions;
+import org.jetbrains.annotations.NotNull;
+
+@SuppressWarnings("unused")
+public class LocalDataSource extends CassandraDataSource
+{
+    @Override
+    @NotNull
+    public String shortName()
+    {
+        return "localsstabledatasource";
+    }
+
+    @Override
+    @NotNull
+    public DataLayer getDataLayer(@NotNull DataSourceOptions options)
+    {
+        // options.asMap() returns the keyLowerCasedMap, therefore all the keys need to be lower-cased
+        return LocalDataLayer.from(options.asMap());
+    }
+}
diff --git a/cassandra-analytics-core/src/main/spark2/org/apache/cassandra/spark/sparksql/SparkRowIterator.java b/cassandra-analytics-core/src/main/spark2/org/apache/cassandra/spark/sparksql/SparkRowIterator.java
new file mode 100644
index 0000000..6bab674
--- /dev/null
+++ b/cassandra-analytics-core/src/main/spark2/org/apache/cassandra/spark/sparksql/SparkRowIterator.java
@@ -0,0 +1,77 @@
+/*
+ * 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.cassandra.spark.sparksql;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import org.apache.cassandra.spark.config.SchemaFeature;
+import org.apache.cassandra.spark.data.DataLayer;
+import org.apache.cassandra.spark.sparksql.filters.CdcOffsetFilter;
+import org.apache.cassandra.spark.sparksql.filters.PartitionKeyFilter;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.sources.v2.reader.InputPartitionReader;
+import org.apache.spark.sql.types.StructType;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Wrapper iterator around SparkCellIterator to normalize cells into Spark SQL rows
+ */
+public class SparkRowIterator extends AbstractSparkRowIterator implements InputPartitionReader<InternalRow>
+{
+    @VisibleForTesting
+    public SparkRowIterator(int partitionId, @NotNull DataLayer dataLayer)
+    {
+        this(partitionId, dataLayer, null, new ArrayList<>());
+    }
+
+    protected SparkRowIterator(int partitionId,
+                               @NotNull DataLayer dataLayer,
+                               @Nullable StructType requiredSchema,
+                               @NotNull List<PartitionKeyFilter> partitionKeyFilters)
+    {
+        this(partitionId, dataLayer, requiredSchema, partitionKeyFilters, null);
+    }
+
+    protected SparkRowIterator(int partitionId,
+                               @NotNull DataLayer dataLayer,
+                               @Nullable StructType columnFilter,
+                               @NotNull List<PartitionKeyFilter> partitionKeyFilters,
+                               @Nullable CdcOffsetFilter cdcOffsetFilter)
+    {
+        super(partitionId, dataLayer, columnFilter, partitionKeyFilters, cdcOffsetFilter);
+    }
+
+    @Override
+    @NotNull
+    RowBuilder newBuilder()
+    {
+        RowBuilder builder = new FullRowBuilder(cqlTable, noValueColumns);
+        for (SchemaFeature feature : requestedFeatures)
+        {
+            builder = feature.decorate(builder);
+        }
+        builder.reset();
+        return builder;
+    }
+}
diff --git a/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/CassandraDataSource.java b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/CassandraDataSource.java
new file mode 100644
index 0000000..b47735a
--- /dev/null
+++ b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/CassandraDataSource.java
@@ -0,0 +1,47 @@
+/*
+ * 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.cassandra.spark.sparksql;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import org.apache.cassandra.spark.data.CassandraDataLayer;
+import org.apache.cassandra.spark.data.CassandraDataSourceHelper;
+import org.apache.cassandra.spark.data.DataLayer;
+
+public class CassandraDataSource extends CassandraTableProvider
+{
+    @Override
+    public String shortName()
+    {
+        return "cassandraBulkRead";
+    }
+
+    @Override
+    public DataLayer getDataLayer(org.apache.spark.sql.util.CaseInsensitiveStringMap options)
+    {
+        return CassandraDataSourceHelper.getDataLayer(options, this::initializeDataLayer);
+    }
+
+    @VisibleForTesting
+    protected void initializeDataLayer(CassandraDataLayer dataLayer, CassandraDataLayer.ClientConfig config)
+    {
+        dataLayer.initialize(config);
+    }
+}
diff --git a/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/CassandraInputPartition.java b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/CassandraInputPartition.java
new file mode 100644
index 0000000..ea76f9b
--- /dev/null
+++ b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/CassandraInputPartition.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.cassandra.spark.sparksql;
+
+import org.apache.spark.sql.connector.read.InputPartition;
+
+class CassandraInputPartition implements InputPartition
+{
+    private final int partitionId;
+
+    CassandraInputPartition(int partitionId)
+    {
+        this.partitionId = partitionId;
+    }
+
+    public int getPartitionId()
+    {
+        return partitionId;
+    }
+}
diff --git a/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/CassandraMicroBatchStream.java b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/CassandraMicroBatchStream.java
new file mode 100644
index 0000000..bc07304
--- /dev/null
+++ b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/CassandraMicroBatchStream.java
@@ -0,0 +1,147 @@
+/*
+ * 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.cassandra.spark.sparksql;
+
+import java.io.Serializable;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.IntStream;
+
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.spark.cdc.CdcRowIterator;
+import org.apache.cassandra.spark.data.DataLayer;
+import org.apache.cassandra.spark.sparksql.filters.CdcOffset;
+import org.apache.cassandra.spark.sparksql.filters.CdcOffsetFilter;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.read.InputPartition;
+import org.apache.spark.sql.connector.read.PartitionReader;
+import org.apache.spark.sql.connector.read.PartitionReaderFactory;
+import org.apache.spark.sql.connector.read.streaming.MicroBatchStream;
+import org.apache.spark.sql.connector.read.streaming.Offset;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.sql.util.CaseInsensitiveStringMap;
+import org.jetbrains.annotations.NotNull;
+
+class CassandraMicroBatchStream implements MicroBatchStream, Serializable
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(CassandraMicroBatchStream.class);
+    private static final int DEFAULT_MIN_MUTATION_AGE_SECS = 0;
+    private static final int DEFAULT_MAX_MUTATION_AGE_SECS = 300;
+
+    private final DataLayer dataLayer;
+    private final StructType requiredSchema;
+    private final long minAgeMicros;
+    private final long maxAgeMicros;
+    private final CdcOffset initial;
+    private CdcOffset start;
+    private CdcOffset end;
+
+    CassandraMicroBatchStream(DataLayer dataLayer,
+                              StructType requiredSchema,
+                              CaseInsensitiveStringMap options)
+    {
+        this.dataLayer = dataLayer;
+        this.requiredSchema = requiredSchema;
+        this.minAgeMicros = TimeUnit.SECONDS.toMicros(options.getLong("minMutationAgeSeconds", DEFAULT_MIN_MUTATION_AGE_SECS));
+        this.maxAgeMicros = TimeUnit.SECONDS.toMicros(options.getLong("maxMutationAgeSeconds", DEFAULT_MAX_MUTATION_AGE_SECS));
+        long nowMicros = nowMicros();
+        this.initial = start(nowMicros);
+        this.start = initial;
+        this.end = end(nowMicros);
+    }
+
+    private long nowMicros()
+    {
+        return TimeUnit.MILLISECONDS.toMicros(System.currentTimeMillis());
+    }
+
+    @NotNull
+    private CdcOffset start(long nowMicros)
+    {
+        return new CdcOffset(nowMicros - maxAgeMicros);
+    }
+
+    @NotNull
+    private CdcOffset end(long nowMicros)
+    {
+        return new CdcOffset(nowMicros - minAgeMicros);
+    }
+
+    public Offset initialOffset()
+    {
+        return initial;
+    }
+
+    public Offset latestOffset()
+    {
+        return end(nowMicros());
+    }
+
+    public Offset deserializeOffset(String json)
+    {
+        return CdcOffset.fromJson(json);
+    }
+
+    public void commit(Offset end)
+    {
+        LOGGER.info("Commit CassandraMicroBatchStream end end='{}'", end);
+    }
+
+    public void stop()
+    {
+        LOGGER.info("Stopping CassandraMicroBatchStream start='{}' end='{}'", start, end);
+    }
+
+    @Override
+    public InputPartition[] planInputPartitions(Offset start, Offset end)
+    {
+        this.start = (CdcOffset) start;
+        this.end = (CdcOffset) end;
+        int numPartitions = dataLayer.partitionCount();
+        LOGGER.info("Planning CDC input partitions numPartitions={} start='{}' end='{}'", numPartitions, start, end);
+        return IntStream.range(0, numPartitions)
+                        .mapToObj(CassandraInputPartition::new)
+                        .toArray(InputPartition[]::new);
+    }
+
+    public PartitionReaderFactory createReaderFactory()
+    {
+        return this::createCdcRowIteratorForPartition;
+    }
+
+    private PartitionReader<InternalRow> createCdcRowIteratorForPartition(InputPartition partition)
+    {
+        Preconditions.checkNotNull(partition, "Null InputPartition");
+        if (partition instanceof CassandraInputPartition)
+        {
+            CassandraInputPartition cassandraInputPartition = (CassandraInputPartition) partition;
+            Preconditions.checkNotNull(start, "Start offset was not set");
+            LOGGER.info("Opening CdcRowIterator start='{}' end='{}' partitionId={}",
+                        start.getTimestampMicros(), end.getTimestampMicros(), cassandraInputPartition.getPartitionId());
+            return new CdcRowIterator(cassandraInputPartition.getPartitionId(),
+                                      dataLayer,
+                                      requiredSchema,
+                                      CdcOffsetFilter.of(start, dataLayer.cdcWatermarkWindow()));
+        }
+        throw new UnsupportedOperationException("Unexpected InputPartition type: " + (partition.getClass().getName()));
+    }
+}
diff --git a/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/CassandraPartitionReaderFactory.java b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/CassandraPartitionReaderFactory.java
new file mode 100644
index 0000000..b2a7df7
--- /dev/null
+++ b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/CassandraPartitionReaderFactory.java
@@ -0,0 +1,69 @@
+/*
+ * 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.cassandra.spark.sparksql;
+
+import java.util.List;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.spark.data.DataLayer;
+import org.apache.cassandra.spark.sparksql.filters.PartitionKeyFilter;
+import org.apache.spark.TaskContext;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.read.InputPartition;
+import org.apache.spark.sql.connector.read.PartitionReader;
+import org.apache.spark.sql.connector.read.PartitionReaderFactory;
+import org.apache.spark.sql.types.StructType;
+
+class CassandraPartitionReaderFactory implements PartitionReaderFactory
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(CassandraPartitionReaderFactory.class);
+    final DataLayer dataLayer;
+    final StructType requiredSchema;
+    final List<PartitionKeyFilter> partitionKeyFilters;
+
+    CassandraPartitionReaderFactory(DataLayer dataLayer,
+                                    StructType requiredSchema,
+                                    List<PartitionKeyFilter> partitionKeyFilters)
+    {
+        this.dataLayer = dataLayer;
+        this.requiredSchema = requiredSchema;
+        this.partitionKeyFilters = partitionKeyFilters;
+    }
+
+    @Override
+    public PartitionReader<InternalRow> createReader(InputPartition partition)
+    {
+        int partitionId;
+        if (partition instanceof CassandraInputPartition)
+        {
+            partitionId = ((CassandraInputPartition) partition).getPartitionId();
+        }
+        else
+        {
+            partitionId = TaskContext.getPartitionId();
+            LOGGER.warn("InputPartition is not of CassandraInputPartition type. "
+                      + "Using TaskContext to determine the partitionId type={}, partitionId={}",
+                        partition.getClass().getName(), partitionId);
+        }
+        return new SparkRowIterator(partitionId, dataLayer, requiredSchema, partitionKeyFilters);
+    }
+}
diff --git a/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/CassandraPartitioning.java b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/CassandraPartitioning.java
new file mode 100644
index 0000000..ac8d687
--- /dev/null
+++ b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/CassandraPartitioning.java
@@ -0,0 +1,60 @@
+/*
+ * 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.cassandra.spark.sparksql;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import org.apache.cassandra.spark.data.CqlField;
+import org.apache.cassandra.spark.data.DataLayer;
+import org.apache.spark.sql.connector.read.partitioning.ClusteredDistribution;
+import org.apache.spark.sql.connector.read.partitioning.Distribution;
+import org.apache.spark.sql.connector.read.partitioning.Partitioning;
+
+class CassandraPartitioning implements Partitioning
+{
+    final DataLayer dataLayer;
+
+    CassandraPartitioning(DataLayer dataLayer)
+    {
+        this.dataLayer = dataLayer;
+    }
+
+    @Override
+    public int numPartitions()
+    {
+        return dataLayer.partitionCount();
+    }
+
+    @Override
+    public boolean satisfy(Distribution distribution)
+    {
+        if (distribution instanceof ClusteredDistribution)
+        {
+            String[] clusteredCols = ((ClusteredDistribution) distribution).clusteredColumns;
+            List<String> partitionKeys = dataLayer.cqlTable().partitionKeys().stream()
+                                                                             .map(CqlField::name)
+                                                                             .collect(Collectors.toList());
+            return Arrays.asList(clusteredCols).containsAll(partitionKeys);
+        }
+        return false;
+    }
+}
diff --git a/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/CassandraScanBuilder.java b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/CassandraScanBuilder.java
new file mode 100644
index 0000000..df2a477
--- /dev/null
+++ b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/CassandraScanBuilder.java
@@ -0,0 +1,156 @@
+/*
+ * 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.cassandra.spark.sparksql;
+
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import org.apache.cassandra.spark.data.CqlField;
+import org.apache.cassandra.spark.data.DataLayer;
+import org.apache.cassandra.spark.sparksql.filters.PartitionKeyFilter;
+import org.apache.cassandra.spark.utils.FilterUtils;
+import org.apache.spark.sql.connector.read.Batch;
+import org.apache.spark.sql.connector.read.InputPartition;
+import org.apache.spark.sql.connector.read.PartitionReaderFactory;
+import org.apache.spark.sql.connector.read.Scan;
+import org.apache.spark.sql.connector.read.ScanBuilder;
+import org.apache.spark.sql.connector.read.SupportsPushDownFilters;
+import org.apache.spark.sql.connector.read.SupportsPushDownRequiredColumns;
+import org.apache.spark.sql.connector.read.SupportsReportPartitioning;
+import org.apache.spark.sql.connector.read.partitioning.Partitioning;
+import org.apache.spark.sql.connector.read.streaming.MicroBatchStream;
+import org.apache.spark.sql.sources.Filter;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.sql.util.CaseInsensitiveStringMap;
+
+class CassandraScanBuilder implements ScanBuilder, Scan, Batch, SupportsPushDownFilters, SupportsPushDownRequiredColumns, SupportsReportPartitioning
+{
+    final DataLayer dataLayer;
+    final StructType schema;
+    final CaseInsensitiveStringMap options;
+    StructType requiredSchema = null;
+    Filter[] pushedFilters = new Filter[0];
+
+    CassandraScanBuilder(DataLayer dataLayer, StructType schema, CaseInsensitiveStringMap options)
+    {
+        this.dataLayer = dataLayer;
+        this.schema = schema;
+        this.options = options;
+    }
+
+    @Override
+    public Scan build()
+    {
+        return this;
+    }
+
+    @Override
+    public void pruneColumns(StructType requiredSchema)
+    {
+        this.requiredSchema = requiredSchema;
+    }
+
+    @Override
+    public Filter[] pushFilters(Filter[] filters)
+    {
+        Filter[] unsupportedFilters = dataLayer.unsupportedPushDownFilters(filters);
+
+        List<Filter> supportedFilters = new ArrayList<>(Arrays.asList(filters));
+        supportedFilters.removeAll(Arrays.asList(unsupportedFilters));
+        pushedFilters = supportedFilters.toArray(new Filter[0]);
+
+        return unsupportedFilters;
+    }
+
+    @Override
+    public Filter[] pushedFilters()
+    {
+        return pushedFilters;
+    }
+
+    @Override
+    public StructType readSchema()
+    {
+        return requiredSchema;
+    }
+
+    @Override
+    public Batch toBatch()
+    {
+        return this;
+    }
+
+    @Override
+    public InputPartition[] planInputPartitions()
+    {
+        return IntStream.range(0, dataLayer.partitionCount())
+                .mapToObj(CassandraInputPartition::new)
+                .toArray(InputPartition[]::new);
+    }
+
+    @Override
+    public MicroBatchStream toMicroBatchStream(String checkpointLocation)
+    {
+        return new CassandraMicroBatchStream(dataLayer, requiredSchema, options);
+    }
+
+    @Override
+    public PartitionReaderFactory createReaderFactory()
+    {
+        return new CassandraPartitionReaderFactory(dataLayer, requiredSchema, buildPartitionKeyFilters());
+    }
+
+    @Override
+    public Partitioning outputPartitioning()
+    {
+        return new CassandraPartitioning(dataLayer);
+    }
+
+    private List<PartitionKeyFilter> buildPartitionKeyFilters()
+    {
+        List<String> partitionKeyColumnNames = dataLayer.cqlTable().partitionKeys().stream().map(CqlField::name).collect(Collectors.toList());
+        Map<String, List<String>> partitionKeyValues = FilterUtils.extractPartitionKeyValues(pushedFilters, new HashSet<>(partitionKeyColumnNames));
+        if (partitionKeyValues.size() > 0)
+        {
+            List<List<String>> orderedValues = partitionKeyColumnNames.stream().map(partitionKeyValues::get).collect(Collectors.toList());
+            return FilterUtils.cartesianProduct(orderedValues).stream()
+                .map(this::buildFilter)
+                .collect(Collectors.toList());
+        }
+        else
+        {
+            return new ArrayList<>();
+        }
+    }
+
+    private PartitionKeyFilter buildFilter(List<String> keys)
+    {
+        AbstractMap.SimpleEntry<ByteBuffer, BigInteger> filterKey = dataLayer.bridge().getPartitionKey(dataLayer.cqlTable(), dataLayer.partitioner(), keys);
+        return PartitionKeyFilter.create(filterKey.getKey(), filterKey.getValue());
+    }
+}
diff --git a/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/CassandraTable.java b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/CassandraTable.java
new file mode 100644
index 0000000..f78e4c5
--- /dev/null
+++ b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/CassandraTable.java
@@ -0,0 +1,69 @@
+/*
+ * 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.cassandra.spark.sparksql;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import com.google.common.collect.ImmutableList;
+
+import org.apache.cassandra.spark.data.DataLayer;
+import org.apache.spark.sql.connector.catalog.SupportsRead;
+import org.apache.spark.sql.connector.catalog.Table;
+import org.apache.spark.sql.connector.catalog.TableCapability;
+import org.apache.spark.sql.connector.read.ScanBuilder;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.sql.util.CaseInsensitiveStringMap;
+
+class CassandraTable implements Table, SupportsRead
+{
+    private final DataLayer dataLayer;
+    private final StructType schema;
+
+    CassandraTable(DataLayer dataLayer, StructType schema)
+    {
+        this.dataLayer = dataLayer;
+        this.schema = schema;
+    }
+
+    @Override
+    public String name()
+    {
+        return dataLayer.cqlTable().keyspace() + "." + dataLayer.cqlTable().table();
+    }
+
+    @Override
+    public StructType schema()
+    {
+        return schema;
+    }
+
+    @Override
+    public Set<TableCapability> capabilities()
+    {
+        return new HashSet<>(ImmutableList.of(TableCapability.BATCH_READ, TableCapability.MICRO_BATCH_READ));
+    }
+
+    @Override
+    public ScanBuilder newScanBuilder(CaseInsensitiveStringMap options)
+    {
+        return new CassandraScanBuilder(dataLayer, schema, options);
+    }
+}
diff --git a/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/CassandraTableProvider.java b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/CassandraTableProvider.java
new file mode 100644
index 0000000..8134686
--- /dev/null
+++ b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/CassandraTableProvider.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.cassandra.spark.sparksql;
+
+import java.util.Map;
+
+import org.apache.cassandra.bridge.CassandraBridgeFactory;
+import org.apache.cassandra.spark.data.DataLayer;
+import org.apache.spark.sql.connector.catalog.Table;
+import org.apache.spark.sql.connector.catalog.TableProvider;
+import org.apache.spark.sql.connector.expressions.Transform;
+import org.apache.spark.sql.sources.DataSourceRegister;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.sql.util.CaseInsensitiveStringMap;
+
+public abstract class CassandraTableProvider implements TableProvider, DataSourceRegister
+{
+    private DataLayer dataLayer;
+
+    public CassandraTableProvider()
+    {
+        CassandraBridgeFactory.validateBridges();
+    }
+
+    public abstract DataLayer getDataLayer(CaseInsensitiveStringMap options);
+
+    DataLayer getDataLayerInternal(CaseInsensitiveStringMap options)
+    {
+        DataLayer dataLayer = this.dataLayer;
+        if (dataLayer != null)
+        {
+            return dataLayer;
+        }
+        dataLayer = getDataLayer(options);
+        this.dataLayer = dataLayer;
+        return dataLayer;
+    }
+
+    @Override
+    public StructType inferSchema(CaseInsensitiveStringMap options)
+    {
+        return getDataLayerInternal(options).structType();
+    }
+
+    @Override
+    public Table getTable(StructType schema, Transform[] partitioning, Map<String, String> properties)
+    {
+        return new CassandraTable(getDataLayerInternal(new CaseInsensitiveStringMap(properties)), schema);
+    }
+}
diff --git a/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/LocalDataSource.java b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/LocalDataSource.java
new file mode 100644
index 0000000..c9d14e8
--- /dev/null
+++ b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/LocalDataSource.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.cassandra.spark.sparksql;
+
+import org.apache.cassandra.spark.data.DataLayer;
+import org.apache.cassandra.spark.data.LocalDataLayer;
+import org.apache.spark.sql.util.CaseInsensitiveStringMap;
+import org.jetbrains.annotations.NotNull;
+
+@SuppressWarnings("unused")
+public class LocalDataSource extends CassandraTableProvider
+{
+    @Override
+    @NotNull
+    public String shortName()
+    {
+        return "localsstabledatasource";
+    }
+
+    @Override
+    @NotNull
+    public DataLayer getDataLayer(@NotNull CaseInsensitiveStringMap options)
+    {
+        return LocalDataLayer.from(options);
+    }
+}
diff --git a/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/SparkRowIterator.java b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/SparkRowIterator.java
new file mode 100644
index 0000000..f457c3b
--- /dev/null
+++ b/cassandra-analytics-core/src/main/spark3/org/apache/cassandra/spark/sparksql/SparkRowIterator.java
@@ -0,0 +1,198 @@
+/*
+ * 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.cassandra.spark.sparksql;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import org.apache.cassandra.spark.config.SchemaFeature;
+import org.apache.cassandra.spark.data.CqlField;
+import org.apache.cassandra.spark.data.CqlTable;
+import org.apache.cassandra.spark.data.DataLayer;
+import org.apache.cassandra.spark.sparksql.filters.CdcOffsetFilter;
+import org.apache.cassandra.spark.sparksql.filters.PartitionKeyFilter;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.read.PartitionReader;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Wrapper iterator around SparkCellIterator to normalize cells into Spark SQL rows
+ */
+public class SparkRowIterator extends AbstractSparkRowIterator implements PartitionReader<InternalRow>
+{
+    @VisibleForTesting
+    public SparkRowIterator(int partitionId, @NotNull DataLayer dataLayer)
+    {
+        this(partitionId, dataLayer, null, new ArrayList<>(), null);
+    }
+
+    public SparkRowIterator(int partitionId,
+                            @NotNull DataLayer dataLayer,
+                            @Nullable StructType columnFilter,
+                            @NotNull List<PartitionKeyFilter> partitionKeyFilters)
+    {
+        this(partitionId, dataLayer, columnFilter, partitionKeyFilters, null);
+    }
+
+    protected SparkRowIterator(int partitionId,
+                               @NotNull DataLayer dataLayer,
+                               @Nullable StructType columnFilter,
+                               @NotNull List<PartitionKeyFilter> partitionKeyFilters,
+                               @Nullable CdcOffsetFilter cdcOffsetFilter)
+    {
+        super(partitionId, dataLayer, columnFilter, partitionKeyFilters, cdcOffsetFilter);
+    }
+
+    @Override
+    @NotNull
+    RowBuilder newBuilder()
+    {
+        RowBuilder builder;
+        String[] fieldNames = null;
+        if (columnFilter != null)
+        {
+            builder = new PartialRowBuilder(columnFilter, cqlTable, noValueColumns);
+            fieldNames = columnFilter.fieldNames();
+        }
+        else
+        {
+            builder = new FullRowBuilder(cqlTable, noValueColumns);
+        }
+
+        for (SchemaFeature feature : requestedFeatures)
+        {
+            // Only decorate when there is no column filter or when the field is requested in the query,
+            // otherwise we skip decoration
+            if (columnFilter == null || Arrays.stream(fieldNames).anyMatch(feature.fieldName()::equals))
+            {
+                builder = feature.decorate(builder);
+            }
+        }
+
+        builder.reset();
+        return builder;
+    }
+
+    /**
+     * PartialRowBuilder that builds row only containing fields in requiredSchema prune-column filter
+     * NOTE: Spark 3 changed the contract from Spark 2 and requires us to only return the columns specified in
+     * the requiredSchema 'prune column' filter and not a sparse Object[] array with null values for excluded columns
+     */
+    static class PartialRowBuilder extends FullRowBuilder
+    {
+        private final int[] positionsMap;
+        private final boolean hasAllNonValueColumns;
+        private final StructType requiredSchema;
+
+        PartialRowBuilder(@NotNull StructType requiredSchema,
+                          CqlTable table,
+                          boolean noValueColumns)
+        {
+            super(table, noValueColumns);
+            this.requiredSchema = requiredSchema;
+            Set<String> requiredColumns = Arrays.stream(requiredSchema.fields())
+                                                .map(StructField::name)
+                                                .collect(Collectors.toSet());
+            hasAllNonValueColumns = table.fields().stream()
+                                                   .filter(CqlField::isNonValueColumn)
+                                                   .map(CqlField::name)
+                                                   .allMatch(requiredColumns::contains);
+
+            // Map original column position to new position in requiredSchema
+            positionsMap = IntStream.range(0, table.numFields())
+                                    .map(position -> -1)
+                                    .toArray();
+            int position = 0;
+            for (StructField structField : requiredSchema.fields())
+            {
+                CqlField field = table.getField(structField.name());
+                if (field != null)  // Field might be last modified timestamp
+                {
+                    positionsMap[field.position()] = position++;
+                }
+            }
+        }
+
+        @Override
+        public void reset()
+        {
+            count = 0;
+            int totalColumns = requiredSchema.size();
+            if (totalColumns > 0)
+            {
+                result = new Object[totalColumns];
+            }
+            else
+            {
+                result = EMPTY_RESULT;
+            }
+        }
+
+        @Override
+        public int fieldIndex(String name)
+        {
+            return requiredSchema != null ? requiredSchema.fieldIndex(name) : super.fieldIndex(name);
+        }
+
+        @Override
+        public void copyKeys(Cell cell)
+        {
+            if (hasAllNonValueColumns)
+            {
+                // Optimization if we are returning all primary key/static columns we can use the super method
+                super.copyKeys(cell);
+                return;
+            }
+
+            // Otherwise we need to only return columns requested and map to new position in result array
+            int length = noValueColumns || cell.isTombstone() ? cell.values.length : cell.values.length - 1;
+            for (int index = 0; index < length; index++)
+            {
+                int position = positionsMap[index];
+                if (position >= 0)
+                {
+                    result[position] = cell.values[index];
+                }
+            }
+            count += length;
+        }
+
+        @Override
+        public void copyValue(Cell cell)
+        {
+            // Copy the next value column mapping column to new position
+            int position = positionsMap[cell.position];
+            if (position >= 0)
+            {
+                result[position] = cell.values[cell.values.length - 1];
+            }
+            count++;
+        }
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/bridge/CassandraVersionFeaturesTest.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/bridge/CassandraVersionFeaturesTest.java
new file mode 100644
index 0000000..e6b8a24
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/bridge/CassandraVersionFeaturesTest.java
@@ -0,0 +1,59 @@
+/*
+ * 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.cassandra.bridge;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class CassandraVersionFeaturesTest
+{
+    @Test
+    public void testCassandraVersions()
+    {
+        testCassandraVersion("1.2.3", 12, 3, "");
+        testCassandraVersion("1.2.3.4", 12, 3, "4");
+
+        testCassandraVersion("cassandra-2.0.14-v3", 20, 14, "-v3");
+        testCassandraVersion("cassandra-1.2.11-v1", 12, 11, "-v1");
+        testCassandraVersion("cassandra-1.2.11.2-tag", 12, 11, "2");
+        testCassandraVersion("cassandra-4.0-SNAPSHOT", 40, 0, "SNAPSHOT");
+        testCassandraVersion("cassandra-2.0.9-loadtest-SNAPSHOT", 20, 9, "-loadtest-SNAPSHOT");
+
+        testCassandraVersion("qwerty-cassandra-1.2.11-v1", 12, 11, "-v1");
+        testCassandraVersion("qwerty-cassandra-1.2.11.2-tag", 12, 11, "2");
+        testCassandraVersion("qwerty-cassandra-4.0-SNAPSHOT", 40, 0, "SNAPSHOT");
+    }
+
+    @Test(expected = RuntimeException.class)
+    public void testInvalidInput()
+    {
+        CassandraVersionFeatures.cassandraVersionFeaturesFromCassandraVersion("qwerty");
+    }
+
+    private static void testCassandraVersion(String version, int major, int minor, String suffix)
+    {
+        CassandraVersionFeatures features = CassandraVersionFeatures.cassandraVersionFeaturesFromCassandraVersion(version);
+
+        assertEquals("Wrong major version for " + version + ",", major, features.getMajorVersion());
+        assertEquals("Wrong minor version for " + version + ",", minor, features.getMinorVersion());
+        assertEquals("Wrong version suffix for " + version + ",", suffix, features.getSuffix());
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/bridge/CompressionTests.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/bridge/CompressionTests.java
new file mode 100644
index 0000000..24c8dce
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/bridge/CompressionTests.java
@@ -0,0 +1,65 @@
+/*
+ * 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.cassandra.bridge;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+import org.junit.Test;
+
+import org.apache.cassandra.spark.utils.RandomUtils;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+public class CompressionTests extends VersionRunner
+{
+    public CompressionTests(CassandraVersion version)
+    {
+        super(version);
+    }
+
+    @Test
+    public void testCompressRandom() throws IOException
+    {
+        // Test with random data - not highly compressible
+        testCompression(RandomUtils.randomBytes(4096));
+    }
+
+    @Test
+    public void testCompressUniform() throws IOException
+    {
+        // Test with highly compressible data
+        byte[] bytes = new byte[4096];
+        Arrays.fill(bytes, (byte) 'a');
+        testCompression(bytes);
+    }
+
+    private void testCompression(byte[] bytes) throws IOException
+    {
+        ByteBuffer compressed = bridge.compress(bytes);
+        ByteBuffer uncompressed = bridge.uncompress(compressed);
+        byte[] result = new byte[uncompressed.remaining()];
+        uncompressed.get(result);
+        assertEquals(bytes.length, result.length);
+        assertArrayEquals(bytes, result);
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/bridge/VersionRunner.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/bridge/VersionRunner.java
new file mode 100644
index 0000000..8d19ada
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/bridge/VersionRunner.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.cassandra.bridge;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.stream.Collectors;
+
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+/**
+ * Runs parameterized tests for all defined versions of Cassandra
+ */
+@RunWith(Parameterized.class)
+public abstract class VersionRunner
+{
+    protected final CassandraVersion version;
+    protected final CassandraBridge bridge;
+
+    @Parameterized.Parameters
+    public static Collection<Object[]> versions()
+    {
+        return Arrays.stream(CassandraVersion.implementedVersions())
+                     .map(version -> new Object[]{version})
+                     .collect(Collectors.toList());
+    }
+
+    public VersionRunner(CassandraVersion version)
+    {
+        this.version = version;
+        this.bridge = CassandraBridgeFactory.get(version);
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/clients/SidecarClientConfigTest.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/clients/SidecarClientConfigTest.java
new file mode 100644
index 0000000..eb218f4
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/clients/SidecarClientConfigTest.java
@@ -0,0 +1,101 @@
+/*
+ * 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.cassandra.clients;
+
+import com.google.common.collect.ImmutableMap;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Unit tests for the {@link Sidecar.ClientConfig} inner class
+ */
+public class SidecarClientConfigTest
+{
+    @Test
+    public void testDefaults()
+    {
+        Sidecar.ClientConfig clientConfig = Sidecar.ClientConfig.create(ImmutableMap.of());
+        assertEquals(9043, clientConfig.port());
+        assertEquals(10, clientConfig.maxRetries());
+        assertEquals(500, clientConfig.millisToSleep());
+        assertEquals(60_000, clientConfig.maxMillisToSleep());
+        assertEquals(6L * 1024L * 1024L, clientConfig.maxBufferSize());
+        assertEquals(4L * 1024L * 1024L, clientConfig.chunkBufferSize());
+        assertEquals(64, clientConfig.maxPoolSize());
+        assertEquals(600, clientConfig.timeoutSeconds());
+    }
+
+    @Test
+    public void testSidecarPort()
+    {
+        Sidecar.ClientConfig clientConfig = Sidecar.ClientConfig.create(ImmutableMap.of("sidecar_port", "9999"));
+        assertEquals(9999, clientConfig.port());
+    }
+
+    @Test
+    public void testMaxRetries()
+    {
+        Sidecar.ClientConfig clientConfig = Sidecar.ClientConfig.create(ImmutableMap.of("maxretries", "5"));
+        assertEquals(5, clientConfig.maxRetries());
+    }
+
+    @Test
+    public void testMillisToSleep()
+    {
+        Sidecar.ClientConfig clientConfig = Sidecar.ClientConfig.create(ImmutableMap.of("defaultmillistosleep", "5000"));
+        assertEquals(5000, clientConfig.millisToSleep());
+    }
+
+    @Test
+    public void testMaxMillisToSleep()
+    {
+        Sidecar.ClientConfig clientConfig = Sidecar.ClientConfig.create(ImmutableMap.of("maxmillistosleep", "30000"));
+        assertEquals(30_000, clientConfig.maxMillisToSleep());
+    }
+
+    @Test
+    public void testMaxBufferSize()
+    {
+        Sidecar.ClientConfig clientConfig = Sidecar.ClientConfig.create(ImmutableMap.of("maxbuffersizebytes", "8"));
+        assertEquals(8, clientConfig.maxBufferSize());
+    }
+
+    @Test
+    public void testChunkBufferSize()
+    {
+        Sidecar.ClientConfig clientConfig = Sidecar.ClientConfig.create(ImmutableMap.of("chunkbuffersizebytes", "24"));
+        assertEquals(24, clientConfig.chunkBufferSize());
+    }
+
+    @Test
+    public void testMaxPoolSize()
+    {
+        Sidecar.ClientConfig clientConfig = Sidecar.ClientConfig.create(ImmutableMap.of("maxpoolsize", "150"));
+        assertEquals(150, clientConfig.maxPoolSize());
+    }
+
+    @Test
+    public void testTimeoutSeconds()
+    {
+        Sidecar.ClientConfig clientConfig = Sidecar.ClientConfig.create(ImmutableMap.of("timeoutseconds", "2"));
+        assertEquals(2, clientConfig.timeoutSeconds());
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/clients/SidecarInstanceImplTest.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/clients/SidecarInstanceImplTest.java
new file mode 100644
index 0000000..ffb1a64
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/clients/SidecarInstanceImplTest.java
@@ -0,0 +1,74 @@
+/*
+ * 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.cassandra.clients;
+
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.esotericsoftware.kryo.io.Output;
+import org.apache.cassandra.sidecar.client.SidecarInstance;
+
+import static org.apache.cassandra.spark.utils.SerializationUtils.deserialize;
+import static org.apache.cassandra.spark.utils.SerializationUtils.kryoDeserialize;
+import static org.apache.cassandra.spark.utils.SerializationUtils.kryoSerialize;
+import static org.apache.cassandra.spark.utils.SerializationUtils.register;
+import static org.apache.cassandra.spark.utils.SerializationUtils.serialize;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+/**
+ * Unit tests for the {@link SidecarInstanceImpl} class
+ */
+public class SidecarInstanceImplTest extends SidecarInstanceTest
+{
+    @BeforeClass
+    public static void setupKryo()
+    {
+        register(SidecarInstanceImpl.class, new SidecarInstanceImpl.Serializer());
+    }
+
+    @Override
+    protected SidecarInstance newInstance(String hostname, int port)
+    {
+        return new SidecarInstanceImpl(hostname, port);
+    }
+
+    @Test
+    public void testJdkSerDe()
+    {
+        SidecarInstance instance = newInstance("localhost", 9043);
+        byte[] bytes = serialize(instance);
+        SidecarInstance deserialized = deserialize(bytes, SidecarInstanceImpl.class);
+        assertNotNull(deserialized);
+        assertEquals("localhost", deserialized.hostname());
+        assertEquals(9043, deserialized.port());
+    }
+
+    @Test
+    public void testKryoSerDe()
+    {
+        SidecarInstance instance = newInstance("localhost", 9043);
+        Output out = kryoSerialize(instance);
+        SidecarInstance deserialized = kryoDeserialize(out, SidecarInstanceImpl.class);
+        assertNotNull(deserialized);
+        assertEquals("localhost", deserialized.hostname());
+        assertEquals(9043, deserialized.port());
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/clients/SidecarInstanceTest.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/clients/SidecarInstanceTest.java
new file mode 100644
index 0000000..d2033ba
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/clients/SidecarInstanceTest.java
@@ -0,0 +1,85 @@
+/*
+ * 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.cassandra.clients;
+
+import org.junit.Test;
+
+import org.apache.cassandra.sidecar.client.SidecarInstance;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.fail;
+
+/**
+ * Abstract class that provides a set of base unit tests for the {@link SidecarInstance} interface
+ */
+public abstract class SidecarInstanceTest
+{
+    protected abstract SidecarInstance newInstance(String hostname, int port);
+
+    @Test
+    public void failsWithInvalidPortNumber()
+    {
+        // Should use Parameterized instead
+        int[] invalidPortNumbers = {-1, 0, 65536 };
+
+        for (int invalidPortNumber : invalidPortNumbers)
+        {
+            try
+            {
+                newInstance(null, invalidPortNumber);
+                fail("Expected to throw AssertionError when port is invalid");
+            }
+            catch (IllegalArgumentException illegalArgumentException)
+            {
+                assertEquals("The Sidecar port number must be in the range 1-65535: " + invalidPortNumber,
+                             illegalArgumentException.getMessage());
+            }
+        }
+    }
+
+    @Test
+    public void failsWithNullHostname()
+    {
+        try
+        {
+            newInstance(null, 8080);
+            fail("Expected to throw NullPointerException when hostname is null");
+        }
+        catch (NullPointerException npe)
+        {
+            assertEquals("The Sidecar hostname must be non-null", npe.getMessage());
+        }
+    }
+
+    @Test
+    public void testConstructorWithValidParameters()
+    {
+        SidecarInstance instance1 = newInstance("localhost", 8080);
+        assertNotNull(instance1);
+        assertEquals(8080, instance1.port());
+        assertEquals("localhost", instance1.hostname());
+
+        SidecarInstance instance2 = newInstance("127.0.0.1", 1234);
+        assertNotNull(instance2);
+        assertEquals(1234, instance2.port());
+        assertEquals("127.0.0.1", instance2.hostname());
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/clients/SslConfigTest.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/clients/SslConfigTest.java
new file mode 100644
index 0000000..d39a3a9
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/clients/SslConfigTest.java
@@ -0,0 +1,216 @@
+/*
+ * 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.cassandra.clients;
+
+import java.util.Collections;
+import java.util.Map;
+import java.util.TreeMap;
+
+import org.junit.Test;
+
+import static org.apache.cassandra.clients.SslConfig.KEYSTORE_BASE64_ENCODED;
+import static org.apache.cassandra.clients.SslConfig.KEYSTORE_PASSWORD;
+import static org.apache.cassandra.clients.SslConfig.KEYSTORE_PATH;
+import static org.apache.cassandra.clients.SslConfig.TRUSTSTORE_BASE64_ENCODED;
+import static org.apache.cassandra.clients.SslConfig.TRUSTSTORE_PASSWORD;
+import static org.apache.cassandra.clients.SslConfig.TRUSTSTORE_PATH;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.fail;
+
+/**
+ * Unit tests for the {@link SslConfig} class
+ */
+public class SslConfigTest
+{
+    @Test
+    public void testEmptyConfiguration()
+    {
+        assertNull(SslConfig.create(Collections.emptyMap()));
+    }
+
+    @Test
+    public void validationFailsWhenBothKeyStorePathAndEncodedKeyStoreAreProvided()
+    {
+        try
+        {
+            new SslConfig.Builder<>()
+            .keyStorePath("/foo")
+            .base64EncodedKeyStore("a")
+            .validate();
+            fail("validation should fail when both key store path and encoded key store are provided");
+        }
+        catch (IllegalArgumentException exception)
+        {
+            assertEquals("Both 'KEYSTORE_PATH' and 'KEYSTORE_BASE64_ENCODED' options were provided. "
+                       + "Only one of the options can be provided", exception.getMessage());
+        }
+    }
+
+    @Test
+    public void validationFailsWhenKeystorePasswordIsNotProvided1()
+    {
+        try
+        {
+            new SslConfig.Builder<>()
+            .keyStorePath("/foo")
+            .validate();
+            fail("validation should fail the keystore path is provided and the keystore password is not provided");
+        }
+        catch (IllegalArgumentException exception)
+        {
+            assertEquals("The 'KEYSTORE_PASSWORD' option must be provided when either the 'KEYSTORE_PATH'"
+                       + " or 'KEYSTORE_BASE64_ENCODED' options are provided", exception.getMessage());
+        }
+    }
+
+    @Test
+    public void validationFailsWhenKeystorePasswordIsNotProvided2()
+    {
+        try
+        {
+            new SslConfig.Builder<>()
+            .base64EncodedKeyStore("a")
+            .validate();
+            fail("validation should fail when the encoded keystore is provided and the keystore password is not provided");
+        }
+        catch (IllegalArgumentException exception)
+        {
+            assertEquals("The 'KEYSTORE_PASSWORD' option must be provided when either the 'KEYSTORE_PATH'"
+                       + " or 'KEYSTORE_BASE64_ENCODED' options are provided", exception.getMessage());
+        }
+    }
+
+    @Test
+    public void validationFailsWhenBothTrustStorePathAndEncodedTrustStoreAreProvided()
+    {
+        try
+        {
+            new SslConfig.Builder<>()
+            .keyStorePath("/foo")
+            .keyStorePassword("pass")
+            .trustStorePath("/bar")
+            .base64EncodedTrustStore("a")
+            .trustStorePassword("pass")
+            .validate();
+            fail("validation should fail when both trust store path and encoded trust store are provided");
+        }
+        catch (IllegalArgumentException exception)
+        {
+            assertEquals("Both 'TRUSTSTORE_PATH' and 'TRUSTSTORE_BASE64_ENCODED' options were provided. "
+                       + "Only one of the options can be provided", exception.getMessage());
+        }
+    }
+
+    @Test
+    public void validationFailsTrustStorePasswordIsNotProvided1()
+    {
+        try
+        {
+            new SslConfig.Builder<>()
+            .keyStorePath("/foo")
+            .keyStorePassword("pass")
+            .trustStorePath("/bar")
+            .validate();
+            fail("validation should fail when trust store path is provided and trust store password is not provided");
+        }
+        catch (IllegalArgumentException exception)
+        {
+            assertEquals("The 'TRUSTSTORE_PASSWORD' option must be provided when either the 'TRUSTSTORE_PATH'"
+                       + " or 'TRUSTSTORE_BASE64_ENCODED' options are provided", exception.getMessage());
+        }
+    }
+
+    @Test
+    public void validationFailsTrustStorePasswordIsNotProvided2()
+    {
+        try
+        {
+            new SslConfig.Builder<>()
+            .keyStorePath("/foo")
+            .keyStorePassword("pass")
+            .base64EncodedTrustStore("a")
+            .validate();
+            fail("validation should fail when encoded trust store is provided and trust store password is not provided");
+        }
+        catch (IllegalArgumentException exception)
+        {
+            assertEquals("The 'TRUSTSTORE_PASSWORD' option must be provided when either the 'TRUSTSTORE_PATH'"
+                       + " or 'TRUSTSTORE_BASE64_ENCODED' options are provided", exception.getMessage());
+        }
+    }
+
+    @Test
+    public void buildWithKeyStorePathAndPassword()
+    {
+        Map<String, String> options = new TreeMap<>(String.CASE_INSENSITIVE_ORDER);
+        options.put(KEYSTORE_PATH, "/foo");
+        options.put(KEYSTORE_PASSWORD, "pass");
+        SslConfig config = SslConfig.create(options);
+        assertNotNull(config);
+        assertEquals("/foo", config.keyStorePath());
+        assertEquals("pass", config.keyStorePassword());
+    }
+
+    @Test
+    public void buildWithEncodedKeyStoreAndPassword()
+    {
+        Map<String, String> options = new TreeMap<>(String.CASE_INSENSITIVE_ORDER);
+        options.put(KEYSTORE_BASE64_ENCODED, "AA");
+        options.put(KEYSTORE_PASSWORD, "pass");
+        SslConfig config = SslConfig.create(options);
+        assertNotNull(config);
+        assertEquals("AA", config.base64EncodedKeyStore());
+        assertEquals("pass", config.keyStorePassword());
+    }
+
+    @Test
+    public void buildWithTrustStorePathAndPassword()
+    {
+        Map<String, String> options = new TreeMap<>(String.CASE_INSENSITIVE_ORDER);
+        options.put(KEYSTORE_PATH, "/foo");
+        options.put(KEYSTORE_PASSWORD, "pass");
+        options.put(TRUSTSTORE_PATH, "/bar");
+        options.put(TRUSTSTORE_PASSWORD, "passs");
+        SslConfig config = SslConfig.create(options);
+        assertNotNull(config);
+        assertEquals("/foo", config.keyStorePath());
+        assertEquals("pass", config.keyStorePassword());
+        assertEquals("/bar", config.trustStorePath());
+        assertEquals("passs", config.trustStorePassword());
+    }
+
+    @Test
+    public void buildWithEncodedTrustStoreAndPassword()
+    {
+        Map<String, String> options = new TreeMap<>(String.CASE_INSENSITIVE_ORDER);
+        options.put(KEYSTORE_PATH, "/foo");
+        options.put(KEYSTORE_PASSWORD, "pass");
+        options.put(TRUSTSTORE_BASE64_ENCODED, "AAA");
+        options.put(TRUSTSTORE_PASSWORD, "passs");
+        SslConfig config = SslConfig.create(options);
+        assertNotNull(config);
+        assertEquals("/foo", config.keyStorePath());
+        assertEquals("pass", config.keyStorePassword());
+        assertEquals("AAA", config.base64EncodedTrustStore());
+        assertEquals("passs", config.trustStorePassword());
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/EndToEndTests.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/EndToEndTests.java
new file mode 100644
index 0000000..ae1312b
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/EndToEndTests.java
@@ -0,0 +1,2312 @@
+/*
+ * 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.cassandra.spark;
+
+import java.sql.Timestamp;
+import java.time.Instant;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Maps;
+import com.google.common.util.concurrent.Uninterruptibles;
+import org.apache.commons.lang3.mutable.MutableLong;
+import org.apache.commons.lang3.tuple.Pair;
+import org.junit.Test;
+
+import org.apache.cassandra.bridge.CassandraVersion;
+import org.apache.cassandra.spark.data.CqlField;
+import org.apache.cassandra.spark.data.SSTable;
+import org.apache.cassandra.spark.data.VersionRunner;
+import org.apache.cassandra.spark.stats.Stats;
+import org.apache.cassandra.spark.utils.RandomUtils;
+import org.apache.cassandra.spark.utils.streaming.SSTableSource;
+import org.apache.cassandra.spark.utils.test.TestSchema;
+import org.apache.spark.sql.Row;
+import scala.collection.mutable.AbstractSeq;
+
+import static org.apache.cassandra.spark.utils.ScalaConversionUtils.mutableSeqAsJavaList;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.quicktheories.QuickTheory.qt;
+import static org.quicktheories.generators.SourceDSL.booleans;
+import static org.quicktheories.generators.SourceDSL.characters;
+import static org.quicktheories.generators.SourceDSL.integers;
+
+/**
+ * End-to-end tests that write random data to multiple SSTables,
+ * reads the data back into Spark and verifies the rows in Spark match the expected.
+ * Uses QuickTheories to test many combinations of field data types and clustering key sort order.
+ * Uses custom SSTableTombstoneWriter to write SSTables with tombstones
+ * to verify Spark Bulk Reader correctly purges tombstoned data.
+ */
+public class EndToEndTests extends VersionRunner
+{
+    public EndToEndTests(CassandraVersion version)
+    {
+        super(version);
+    }
+
+    /* Partition Key Tests */
+
+    @Test
+    public void testSinglePartitionKey()
+    {
+        Tester.builder(TestSchema.builder()
+                                 .withPartitionKey("pk", bridge.uuid())
+                                 .withColumn("c1", bridge.bigint())
+                                 .withColumn("c2", bridge.text()))
+              .withExpectedRowCountPerSSTable(Tester.DEFAULT_NUM_ROWS)
+              .withSumField("c1")
+              .run();
+    }
+
+    @Test
+    public void testOnlyPartitionKeys()
+    {
+        // Special case where schema is only partition keys
+        Tester.builder(TestSchema.builder()
+                                 .withPartitionKey("a", bridge.uuid()))
+              .withExpectedRowCountPerSSTable(Tester.DEFAULT_NUM_ROWS)
+              .run();
+        Tester.builder(TestSchema.builder()
+                                 .withPartitionKey("a", bridge.uuid())
+                                 .withPartitionKey("b", bridge.bigint()))
+              .withExpectedRowCountPerSSTable(Tester.DEFAULT_NUM_ROWS)
+              .run();
+    }
+
+    @Test
+    public void testOnlyPartitionClusteringKeys()
+    {
+        Tester.builder(TestSchema.builder()
+                                 .withPartitionKey("a", bridge.uuid())
+                                 .withClusteringKey("b", bridge.bigint())
+                                 .withClusteringKey("c", bridge.text()))
+              .withExpectedRowCountPerSSTable(Tester.DEFAULT_NUM_ROWS)
+              .run();
+    }
+
+    @Test
+    public void testMultiplePartitionKeys()
+    {
+        Tester.builder(TestSchema.builder()
+                                 .withPartitionKey("a", bridge.uuid())
+                                 .withPartitionKey("b", bridge.bigint())
+                                 .withColumn("c", bridge.text())
+                                 .withColumn("d", bridge.bigint()))
+              .withExpectedRowCountPerSSTable(Tester.DEFAULT_NUM_ROWS)
+              .withSumField("d")
+              .run();
+    }
+
+    /* Clustering Key Tests */
+
+    @Test
+    public void testBasicSingleClusteringKey()
+    {
+        Tester.builder(TestSchema.builder()
+                                 .withPartitionKey("a", bridge.bigint())
+                                 .withClusteringKey("b", bridge.bigint())
+                                 .withColumn("c", bridge.bigint()))
+              .withExpectedRowCountPerSSTable(Tester.DEFAULT_NUM_ROWS)
+              .withSumField("c")
+              .run();
+    }
+
+    @Test
+    public void testSingleClusteringKeyOrderBy()
+    {
+        qt().forAll(TestUtils.cql3Type(bridge), TestUtils.sortOrder())
+            .checkAssert((clusteringKeyType, sortOrder) ->
+                Tester.builder(TestSchema.builder()
+                                         .withPartitionKey("a", bridge.bigint())
+                                         .withClusteringKey("b", clusteringKeyType)
+                                         .withColumn("c", bridge.bigint())
+                                         .withSortOrder(sortOrder))
+                      .withExpectedRowCountPerSSTable(Tester.DEFAULT_NUM_ROWS)
+                      .run()
+            );
+    }
+
+    @Test
+    public void testMultipleClusteringKeys()
+    {
+        Tester.builder(TestSchema.builder()
+                                 .withPartitionKey("a", bridge.uuid())
+                                 .withClusteringKey("b", bridge.aInt())
+                                 .withClusteringKey("c", bridge.text())
+                                 .withColumn("d", bridge.text())
+                                 .withColumn("e", bridge.bigint()))
+              .withExpectedRowCountPerSSTable(Tester.DEFAULT_NUM_ROWS)
+              .withSumField("e")
+              .run();
+    }
+
+    @Test
+    public void testManyClusteringKeys()
+    {
+        Tester.builder(TestSchema.builder()
+                                 .withPartitionKey("a", bridge.uuid())
+                                 .withClusteringKey("b", bridge.timestamp())
+                                 .withClusteringKey("c", bridge.text())
+                                 .withClusteringKey("d", bridge.uuid())
+                                 .withClusteringKey("e", bridge.aFloat())
+                                 .withColumn("f", bridge.text())
+                                 .withColumn("g", bridge.bigint()))
+              .withExpectedRowCountPerSSTable(Tester.DEFAULT_NUM_ROWS)
+              .withSumField("g")
+              .run();
+    }
+
+    /* Data Type Tests */
+
+    @Test
+    public void testAllDataTypesPartitionKey()
+    {
+        // Test partition key can be read for all data types
+        qt().forAll(TestUtils.cql3Type(bridge))
+            .checkAssert(partitionKeyType -> {
+                // Boolean or empty types have limited cardinality
+                int numRows = partitionKeyType.cardinality(10);
+                Tester.builder(TestSchema.builder()
+                                         .withPartitionKey("a", partitionKeyType)
+                                         .withColumn("b", bridge.bigint()))
+                      .withNumRandomSSTables(1)
+                      .withNumRandomRows(numRows)
+                      .withExpectedRowCountPerSSTable(numRows)
+                      .run();
+            });
+    }
+
+    @Test
+    public void testAllDataTypesValueColumn()
+    {
+        // Test value column can be read for all data types
+        qt().forAll(TestUtils.cql3Type(bridge))
+            .checkAssert(valueType ->
+                Tester.builder(TestSchema.builder()
+                                         .withPartitionKey("a", bridge.bigint())
+                                         .withColumn("b", valueType))
+                      .withNumRandomSSTables(1)
+                      .withExpectedRowCountPerSSTable(Tester.DEFAULT_NUM_ROWS)
+                      .run()
+            );
+    }
+
+    /* Compaction */
+
+    @Test
+    public void testMultipleSSTablesCompaction()
+    {
+        AtomicLong startTotal = new AtomicLong(0);
+        AtomicLong newTotal = new AtomicLong(0);
+        Map<UUID, Long> column1 = new HashMap<>(Tester.DEFAULT_NUM_ROWS);
+        Map<UUID, String> column2 = new HashMap<>(Tester.DEFAULT_NUM_ROWS);
+        Tester.builder(TestSchema.builder()
+                                 .withPartitionKey("pk", bridge.uuid())
+                                 .withColumn("c1", bridge.bigint())
+                                 .withColumn("c2", bridge.text()))
+              .dontWriteRandomData()
+              .withSSTableWriter(writer -> {
+                  for (int row = 0; row < Tester.DEFAULT_NUM_ROWS; row++)
+                  {
+                      UUID pk = UUID.randomUUID();
+                      long c1 = RandomUtils.RANDOM.nextInt(10_000_000);
+                      String c2 = UUID.randomUUID().toString();
+                      startTotal.addAndGet(c1);
+                      column1.put(pk, c1);
+                      column2.put(pk, c2);
+                      writer.write(pk, c1, c2);
+                  }
+              })
+              // Overwrite c1 with new value greater than previous
+              .withSSTableWriter(writer -> {
+                  for (UUID pk : column1.keySet())
+                  {
+                      long newBalance = (long) RandomUtils.RANDOM.nextInt(10_000_000) + column1.get(pk);
+                      assertTrue(newBalance > column1.get(pk));
+                      newTotal.addAndGet(newBalance);
+                      column1.put(pk, newBalance);
+                      writer.write(pk, newBalance, column2.get(pk));
+                  }
+              })
+              .withCheck(dataset -> {
+                  assertTrue(startTotal.get() < newTotal.get());
+                  long sum = 0;
+                  int count = 0;
+                  for (Row row : dataset.collectAsList())
+                  {
+                      UUID pk = UUID.fromString(row.getString(0));
+                      assertEquals(row.getLong(1), column1.get(pk).longValue());
+                      assertEquals(row.getString(2), column2.get(pk));
+                      sum += (long) row.get(1);
+                      count++;
+                  }
+                  assertEquals(Tester.DEFAULT_NUM_ROWS, count);
+                  assertEquals(newTotal.get(), sum);
+              })
+              .withReset(() -> {
+                  startTotal.set(0);
+                  newTotal.set(0);
+                  column1.clear();
+                  column2.clear();
+              });
+    }
+
+    @Test
+    public void testCompaction()
+    {
+        int numRowsColumns = 20;
+        AtomicInteger total = new AtomicInteger(0);
+        Tester.builder(TestSchema.builder()
+                                 .withPartitionKey("a", bridge.aInt())
+                                 .withClusteringKey("b", bridge.aInt())
+                                 .withColumn("c", bridge.aInt()))
+              // Don't write random data
+              .dontWriteRandomData()
+              // Write some SSTables deterministically
+              .withSSTableWriter(writer -> {
+                  for (int row = 0; row < numRowsColumns; row++)
+                  {
+                      for (int column = 0; column < numRowsColumns; column++)
+                      {
+                          writer.write(row, column, 0);
+                      }
+                  }
+              })
+              .withSSTableWriter(writer -> {
+                  for (int row = 0; row < numRowsColumns; row++)
+                  {
+                      for (int column = 0; column < numRowsColumns; column++)
+                      {
+                          writer.write(row, column, 1);
+                      }
+                  }
+              })
+              .withSSTableWriter(writer -> {
+                  for (int row = 0; row < numRowsColumns; row++)
+                  {
+                      for (int column = 0; column < numRowsColumns; column++)
+                      {
+                          int num = column * 500;
+                          total.addAndGet(num);
+                          writer.write(row, column, num);
+                      }
+                  }
+              })
+              .withReadListener(row -> {
+                  // We should have compacted the SSTables to remove duplicate data and tombstones
+                  assert row.getInteger("b") * 500 == row.getInteger("c");
+              })
+              // Verify sums to correct total
+              .withCheck(dataset -> assertEquals(total.get(), dataset.groupBy().sum("c").first().getLong(0)))
+              .withCheck(dataset -> assertEquals(numRowsColumns * numRowsColumns,
+                                                 dataset.groupBy().count().first().getLong(0)))
+              .withReset(() -> total.set(0))
+              .run();
+    }
+
+    @Test
+    public void testSingleClusteringKey()
+    {
+        AtomicLong total = new AtomicLong(0);
+        Map<Integer, MutableLong> testSum = new HashMap<>();
+        Set<Integer> clusteringKeys = ImmutableSet.of(0, 1, 2, 3);
+        for (int clusteringKey : clusteringKeys)
+        {
+            testSum.put(clusteringKey, new MutableLong(0));
+        }
+
+        Tester.builder(TestSchema.builder()
+                                 .withPartitionKey("a", bridge.uuid())
+                                 .withClusteringKey("b", bridge.aInt())
+                                 .withColumn("c", bridge.bigint())
+                                 .withColumn("d", bridge.text()))
+              .dontWriteRandomData()
+              .withSSTableWriter(writer -> {
+                  for (int row = 0; row < Tester.DEFAULT_NUM_ROWS; row++)
+                  {
+                      for (int clusteringKey : clusteringKeys)
+                      {
+                          UUID accountId = UUID.randomUUID();
+                          long balance = RandomUtils.RANDOM.nextInt(10_000_000);
+                          total.addAndGet(balance);
+                          String name = UUID.randomUUID().toString().substring(0, 8);
+                          testSum.get(clusteringKey).add(balance);
+                          writer.write(accountId, clusteringKey, balance, name);
+                      }
+                  }
+              })
+              .withExpectedRowCountPerSSTable(Tester.DEFAULT_NUM_ROWS * clusteringKeys.size())
+              .withCheck(dataset -> {
+                  assertEquals(total.get(), testSum.values().stream().mapToLong(MutableLong::getValue).sum());
+                  long sum = 0;
+                  int count = 0;
+                  for (Row row : dataset.collectAsList())
+                  {
+                      assertNotNull(row.getString(0));
+                      long balance = row.getLong(2);
+                      assertNotNull(row.getString(3));
+                      sum += balance;
+                      count++;
+                  }
+                  assertEquals(total.get(), sum);
+                  assertEquals(Tester.DEFAULT_NUM_ROWS * clusteringKeys.size(), count);
+              })
+              .withCheck(dataset -> {
+                  // Test basic group by matches expected
+                  for (Row row : dataset.groupBy("b").sum("c").collectAsList())
+                  {
+                      assertEquals(testSum.get(row.getInt(0)).getValue().longValue(), row.getLong(1));
+                  }
+              })
+              .withReset(() -> {
+                  total.set(0);
+                  for (int clusteringKey : clusteringKeys)
+                  {
+                      testSum.put(clusteringKey, new MutableLong(0));
+                  }
+              })
+              .run();
+    }
+
+    /* Static Columns */
+
+    @Test
+    public void testOnlyStaticColumn()
+    {
+        Tester.builder(TestSchema.builder()
+                                 .withPartitionKey("a", bridge.uuid())
+                                 .withClusteringKey("b", bridge.bigint())
+                                 .withStaticColumn("c", bridge.aInt()))
+              .withExpectedRowCountPerSSTable(Tester.DEFAULT_NUM_ROWS)
+              .run();
+    }
+
+    @Test
+    @SuppressWarnings("UnstableApiUsage")  // Use of Guava Uninterruptibles
+    public void testStaticColumn()
+    {
+        int numRows = 100;
+        int numColumns = 20;
+        Tester.builder(TestSchema.builder()
+                                 .withPartitionKey("a", bridge.aInt())
+                                 .withClusteringKey("b", bridge.aInt())
+                                 .withStaticColumn("c", bridge.aInt())
+                                 .withColumn("d", bridge.text()))
+              // Don't write random data
+              .dontWriteRandomData()
+              // Write some SSTables deterministically
+              .withSSTableWriter(writer -> {
+                  for (int row = 0; row < numRows; row++)
+                  {
+                      for (int column = 0; column < numColumns; column++)
+                      {
+                          // We need to sleep here to prevent timestamp conflicts confusing the static column value
+                          if (column == numColumns - 1)
+                          {
+                              Uninterruptibles.sleepUninterruptibly(2, TimeUnit.MILLISECONDS);
+                          }
+                          writer.write(row, column, row * column, UUID.randomUUID().toString());
+                      }
+                  }
+              })
+              .withSSTableWriter(writer -> {
+                  for (int row = 0; row < numRows; row++)
+                  {
+                      for (int column = numColumns; column < numColumns * 2; column++)
+                      {
+                          // We need to sleep here to prevent timestamp conflicts confusing the static column value
+                          if (column == numColumns * 2 - 1)
+                          {
+                              Uninterruptibles.sleepUninterruptibly(2, TimeUnit.MILLISECONDS);
+                          }
+
+                          writer.write(row, column, row * column, UUID.randomUUID().toString());
+                      }
+                  }
+              })
+              .withReadListener(row -> {
+                  // Static column should be the last value written
+                  assert row.getInteger("c") == row.getInteger("a") * (numColumns * 2 - 1);
+              })
+              // Verify row count is correct
+              .withCheck(dataset -> assertEquals(numRows * numColumns * 2, dataset.count()))
+              .run();
+    }
+
+    @Test
+    public void testNulledStaticColumns()
+    {
+        int numClusteringKeys = 10;
+        Tester.builder(TestSchema.builder()
+                                 .withPartitionKey("a", bridge.uuid())
+                                 .withClusteringKey("b", bridge.aInt())
+                                 .withStaticColumn("c", bridge.text())
+                                 .withColumn("d", bridge.aInt()))
+              .withNumRandomRows(0)
+              .dontCheckNumSSTables()
+              .withSSTableWriter(writer ->
+                  IntStream.range(0, Tester.DEFAULT_NUM_ROWS)
+                           .forEach(row -> {
+                               UUID pk = UUID.randomUUID();
+                               IntStream.range(0, numClusteringKeys)
+                                        .forEach(clusteringKey ->
+                                                writer.write(pk, clusteringKey, row % 2 == 0 ? null : "Non-null", row));
+                           })
+              )
+              .withReadListener(row -> {
+                  String staticCol = row.isNull("c") ? null : row.getString("c");
+                  if (row.getInteger("d") % 2 == 0)
+                  {
+                      assertNull(staticCol);
+                  }
+                  else
+                  {
+                      assertEquals("Non-null", staticCol);
+                  }
+              })
+              .run();
+    }
+
+    @Test
+    public void testMultipleSSTableCompacted()
+    {
+        TestSchema.Builder schemaBuilder = TestSchema.builder()
+                                                     .withPartitionKey("a", bridge.uuid())
+                                                     .withClusteringKey("b", bridge.aInt())
+                                                     .withClusteringKey("c", bridge.text())
+                                                     .withColumn("d", bridge.text())
+                                                     .withColumn("e", bridge.bigint());
+        AtomicLong total = new AtomicLong(0);
+        Map<UUID, TestSchema.TestRow> rows = new HashMap<>(Tester.DEFAULT_NUM_ROWS);
+        Tester.builder(schemaBuilder)
+              // Don't write random data
+              .dontWriteRandomData()
+              // Write some SSTables with random data
+              .withSSTableWriter(writer -> {
+                  for (int row = 0; row < Tester.DEFAULT_NUM_ROWS; row++)
+                  {
+                      TestSchema schema = schemaBuilder.build();
+                      schema.setCassandraVersion(version);
+                      TestSchema.TestRow testRow = schema.randomRow();
+                      rows.put(testRow.getUUID("a"), testRow);
+                      writer.write(testRow.allValues());
+                  }
+              })
+              // Overwrite rows/cells multiple times in different SSTables
+              // and ensure compaction compacts together correctly
+              .withSSTableWriter(writer -> {
+                  for (TestSchema.TestRow testRow : ImmutableSet.copyOf(rows.values()))
+                  {
+                      // Update rows with new values
+                      TestSchema.TestRow newTestRow = testRow.copy("e", RandomUtils.RANDOM.nextLong())
+                                                             .copy("d", UUID.randomUUID().toString().substring(0, 10));
+                      rows.put(testRow.getUUID("a"), newTestRow);
+                      writer.write(newTestRow.allValues());
+                  }
+              })
+              .withSSTableWriter(writer -> {
+                  for (TestSchema.TestRow testRow : ImmutableSet.copyOf(rows.values()))
+                  {
+                      // Update rows with new values - this should be the final values seen by Spark
+                      TestSchema.TestRow newTestRow = testRow.copy("e", RandomUtils.RANDOM.nextLong())
+                                                             .copy("d", UUID.randomUUID().toString().substring(0, 10));
+                      rows.put(testRow.getUUID("a"), newTestRow);
+                      total.addAndGet(newTestRow.getLong("e"));
+                      writer.write(newTestRow.allValues());
+                  }
+              })
+              // Verify rows returned by Spark match expected
+              .withReadListener(actualRow -> assertTrue(rows.containsKey(actualRow.getUUID("a"))))
+              .withReadListener(actualRow -> assertEquals(rows.get(actualRow.getUUID("a")), actualRow))
+              .withReadListener(actualRow -> assertEquals(rows.get(actualRow.getUUID("a")).getLong("e"),
+                                                                   actualRow.getLong("e")))
+              // Verify Spark aggregations match expected
+              .withCheck(dataset -> assertEquals(total.get(), dataset.groupBy().sum("e").first().getLong(0)))
+              .withCheck(dataset -> assertEquals(rows.size(), dataset.groupBy().count().first().getLong(0)))
+              .withReset(() -> {
+                  total.set(0);
+                  rows.clear();
+              })
+              .run();
+    }
+
+    /* Tombstone Tests */
+
+    @Test
+    public void testPartitionTombstoneInt()
+    {
+        int numRows = 100;
+        int numColumns = 10;
+        qt().withExamples(20)
+            .forAll(integers().between(0, numRows - 2))
+            .checkAssert(deleteRangeStart -> {
+                assert 0 <= deleteRangeStart && deleteRangeStart < numRows;
+                int deleteRangeEnd = deleteRangeStart + RandomUtils.RANDOM.nextInt(numRows - deleteRangeStart - 1) + 1;
+                assert deleteRangeStart < deleteRangeEnd && deleteRangeEnd < numRows;
+
+                Tester.builder(TestSchema.basicBuilder(bridge)
+                                         .withDeleteFields("a ="))
+                      .withVersions(TestUtils.tombstoneTestableVersions())
+                      .dontWriteRandomData()
+                      .withSSTableWriter(writer -> {
+                          for (int row = 0; row < numRows; row++)
+                          {
+                              for (int column = 0; column < numColumns; column++)
+                              {
+                                  writer.write(row, column, column);
+                              }
+                          }
+                      })
+                      .withTombstoneWriter(writer -> {
+                          for (int row = deleteRangeStart; row < deleteRangeEnd; row++)
+                          {
+                              writer.write(row);
+                          }
+                      })
+                      .dontCheckNumSSTables()
+                      .withCheck(dataset -> {
+                          int count = 0;
+                          for (Row row : dataset.collectAsList())
+                          {
+                              int value = row.getInt(0);
+                              assertTrue(0 <= value && value < numRows);
+                              assertTrue(value < deleteRangeStart || value >= deleteRangeEnd);
+                              count++;
+                          }
+                          assertEquals((numRows - (deleteRangeEnd - deleteRangeStart)) * numColumns, count);
+                      })
+                      .run();
+            });
+    }
+
+    @Test
+    public void testRowTombstoneInt()
+    {
+        int numRows = 100;
+        int numColumns = 10;
+        qt().withExamples(20)
+            .forAll(integers().between(0, numColumns - 1))
+            .checkAssert(colNum ->
+                Tester.builder(TestSchema.basicBuilder(bridge)
+                                         .withDeleteFields("a =", "b ="))
+                      .withVersions(TestUtils.tombstoneTestableVersions())
+                      .dontWriteRandomData()
+                      .withSSTableWriter(writer -> {
+                          for (int row = 0; row < numRows; row++)
+                          {
+                              for (int column = 0; column < numColumns; column++)
+                              {
+                                  writer.write(row, column, column);
+                              }
+                          }
+                      })
+                      .withTombstoneWriter(writer -> {
+                          for (int row = 0; row < numRows; row++)
+                          {
+                              writer.write(row, colNum);
+                          }
+                      })
+                      .dontCheckNumSSTables()
+                      .withCheck(dataset -> {
+                          int count = 0;
+                          for (Row row : dataset.collectAsList())
+                          {
+                              int value = row.getInt(0);
+                              assertTrue(row.getInt(0) >= 0 && value < numRows);
+                              assertTrue(row.getInt(1) != colNum);
+                              assertEquals(row.get(1), row.get(2));
+                              count++;
+                          }
+                          assertEquals(numRows * (numColumns - 1), count);
+                      })
+                      .run()
+            );
+    }
+
+    @Test
+    public void testRangeTombstoneInt()
+    {
+        int numRows = 10;
+        int numColumns = 128;
+        qt().withExamples(10)
+            .forAll(integers().between(0, numColumns - 1))
+            .checkAssert(startBound -> {
+                assertTrue(startBound < numColumns);
+                int endBound = startBound + RandomUtils.RANDOM.nextInt(numColumns - startBound);
+                assertTrue(endBound >= startBound && endBound <= numColumns);
+                int numTombstones = endBound - startBound;
+
+                Tester.builder(TestSchema.basicBuilder(bridge)
+                                         .withDeleteFields("a =", "b >=", "b <"))
+                      .withVersions(TestUtils.tombstoneTestableVersions())
+                      .dontWriteRandomData()
+                      .withSSTableWriter(writer -> {
+                          for (int row = 0; row < numRows; row++)
+                          {
+                              for (int column = 0; column < numColumns; column++)
+                              {
+                                  writer.write(row, column, column);
+                              }
+                          }
+                      })
+                      .withTombstoneWriter(writer -> {
+                          for (int row = 0; row < numRows; row++)
+                          {
+                              writer.write(row, startBound, endBound);
+                          }
+                      })
+                      .dontCheckNumSSTables()
+                      .withCheck(dataset -> {
+                          int count = 0;
+                          for (Row row : dataset.collectAsList())
+                          {
+                              // Verify row values exist within correct range with range tombstoned values removed
+                              int value = row.getInt(1);
+                              assertEquals(value, row.getInt(2));
+                              assertTrue(value <= numColumns);
+                              assertTrue(value < startBound || value >= endBound);
+                              count++;
+                          }
+                          assertEquals(numRows * (numColumns - numTombstones), count);
+                      })
+                      .run();
+            });
+    }
+
+    @Test
+    public void testRangeTombstoneString()
+    {
+        int numRows = 10;
+        int numColumns = 128;
+        qt().withExamples(10)
+            .forAll(characters().ascii())
+            .checkAssert(startBound -> {
+                assertTrue(startBound <= numColumns);
+                char endBound = (char) (startBound + RandomUtils.RANDOM.nextInt(numColumns - startBound));
+                assertTrue(endBound >= startBound && endBound <= numColumns);
+                int numTombstones = endBound - startBound;
+
+                Tester.builder(TestSchema.builder()
+                                         .withPartitionKey("a", bridge.aInt())
+                                         .withClusteringKey("b", bridge.text())
+                                         .withColumn("c", bridge.aInt())
+                                         .withDeleteFields("a =", "b >=", "b <"))
+                      .withVersions(TestUtils.tombstoneTestableVersions())
+                      .dontWriteRandomData()
+                      .withSSTableWriter(writer -> {
+                          for (int row = 0; row < numRows; row++)
+                          {
+                              for (int column = 0; column < numColumns; column++)
+                              {
+                                  String value = String.valueOf((char) column);
+                                  writer.write(row, value, column);
+                              }
+                          }
+                      })
+                      .withTombstoneWriter(writer -> {
+                          for (int row = 0; row < numRows; row++)
+                          {
+                              writer.write(row, startBound.toString(), Character.toString(endBound));
+                          }
+                      })
+                      .dontCheckNumSSTables()
+                      .withCheck(dataset -> {
+                          int count = 0;
+                          for (Row row : dataset.collectAsList())
+                          {
+                              // Verify row values exist within correct range with range tombstoned values removed
+                              char character = row.getString(1).charAt(0);
+                              assertTrue(character <= numColumns);
+                              assertTrue(character < startBound || character >= endBound);
+                              count++;
+                          }
+                          assertEquals(numRows * (numColumns - numTombstones), count);
+                      })
+                      .run();
+            });
+    }
+
+    /* Partial Rows: test reading rows with missing columns */
+
+    @Test
+    public void testPartialRow()
+    {
+        Map<UUID, UUID> rows = new HashMap<>();
+        Tester.builder(TestSchema.builder()
+                                 .withPartitionKey("a", bridge.uuid())
+                                 .withColumn("b", bridge.text())
+                                 .withColumn("c", bridge.uuid())
+                                 .withColumn("d", bridge.aInt())
+                                 .withColumn("e", bridge.uuid())
+                                 .withColumn("f", bridge.aInt())
+                                 .withInsertFields("a", "c", "e"))  // Override insert statement to only insert some columns
+              .dontWriteRandomData()
+              .withSSTableWriter(writer -> {
+                  for (int row = 0; row < Tester.DEFAULT_NUM_ROWS; row++)
+                  {
+                      UUID key = UUID.randomUUID();
+                      UUID value = UUID.randomUUID();
+                      rows.put(key, value);
+                      writer.write(key, value, value);
+                  }
+              })
+              .withCheck(dataset -> {
+                  for (Row row : dataset.collectAsList())
+                  {
+                      assertEquals(6, row.size());
+                      UUID key = UUID.fromString(row.getString(0));
+                      UUID value1 = UUID.fromString(row.getString(2));
+                      UUID value2 = UUID.fromString(row.getString(4));
+                      assertTrue(rows.containsKey(key));
+                      assertEquals(rows.get(key), value1);
+                      assertEquals(value2, value1);
+                      assertNull(row.get(1));
+                      assertNull(row.get(3));
+                      assertNull(row.get(5));
+                  }
+              })
+              .withReset(rows::clear)
+              .run();
+    }
+
+    @Test
+    public void testPartialRowClusteringKeys()
+    {
+        Map<String, String> rows = new HashMap<>();
+        Tester.builder(TestSchema.builder()
+                                 .withPartitionKey("a", bridge.uuid())
+                                 .withClusteringKey("b", bridge.uuid())
+                                 .withClusteringKey("c", bridge.uuid())
+                                 .withColumn("d", bridge.text())
+                                 .withColumn("e", bridge.uuid())
+                                 .withColumn("f", bridge.aInt())
+                                 .withColumn("g", bridge.uuid())
+                                 .withColumn("h", bridge.aInt())
+                                 .withInsertFields("a", "b", "c", "e", "g"))  // Override insert statement to only insert some columns
+              .dontWriteRandomData()
+              .withSSTableWriter(writer -> {
+                  for (int row = 0; row < Tester.DEFAULT_NUM_ROWS; row++)
+                  {
+                      UUID a = UUID.randomUUID();
+                      UUID b = UUID.randomUUID();
+                      UUID c = UUID.randomUUID();
+                      UUID e = UUID.randomUUID();
+                      UUID g = UUID.randomUUID();
+                      String key = a + ":" + b + ":" + c;
+                      String value = e + ":" + g;
+                      rows.put(key, value);
+                      writer.write(a, b, c, e, g);
+                  }
+              })
+              .withCheck(dataset -> {
+                  for (Row row : dataset.collectAsList())
+                  {
+                      assertEquals(8, row.size());
+                      String a = row.getString(0);
+                      String b = row.getString(1);
+                      String c = row.getString(2);
+                      String e = row.getString(4);
+                      String g = row.getString(6);
+                      String key = a + ":" + b + ":" + c;
+                      String value = e + ":" + g;
+                      assertTrue(rows.containsKey(key));
+                      assertEquals(rows.get(key), value);
+                      assertNull(row.get(3));
+                      assertNull(row.get(5));
+                      assertNull(row.get(7));
+                  }
+              })
+              .withReset(rows::clear)
+              .run();
+    }
+
+    /* Collections */
+
+    @Test
+    public void testSet()
+    {
+        qt().forAll(TestUtils.cql3Type(bridge))
+            .checkAssert(type ->
+                Tester.builder(TestSchema.builder()
+                                         .withPartitionKey("pk", bridge.uuid())
+                                         .withColumn("a", bridge.set(type)))
+                      .withExpectedRowCountPerSSTable(Tester.DEFAULT_NUM_ROWS)
+                      .run()
+            );
+    }
+
+    @Test
+    public void testList()
+    {
+        qt().forAll(TestUtils.cql3Type(bridge))
+            .checkAssert(type ->
+                Tester.builder(TestSchema.builder()
+                                         .withPartitionKey("pk", bridge.uuid())
+                                         .withColumn("a", bridge.list(type)))
+                      .withExpectedRowCountPerSSTable(Tester.DEFAULT_NUM_ROWS)
+                      .run()
+            );
+    }
+
+    @Test
+    public void testMap()
+    {
+        qt().withExamples(50)  // Limit number of tests otherwise n x n tests takes too long
+            .forAll(TestUtils.cql3Type(bridge), TestUtils.cql3Type(bridge))
+            .checkAssert((keyType, valueType) ->
+                Tester.builder(TestSchema.builder()
+                                         .withPartitionKey("pk", bridge.uuid())
+                                         .withColumn("a", bridge.map(keyType, valueType)))
+                      .withExpectedRowCountPerSSTable(Tester.DEFAULT_NUM_ROWS)
+                      .run()
+            );
+    }
+
+    @Test
+    public void testClusteringKeySet()
+    {
+        Tester.builder(TestSchema.builder()
+                                 .withPartitionKey("pk", bridge.uuid())
+                                 .withClusteringKey("id", bridge.aInt())
+                                 .withColumn("a", bridge.set(bridge.text())))
+              .withExpectedRowCountPerSSTable(Tester.DEFAULT_NUM_ROWS)
+              .run();
+    }
+
+    /* Frozen Collections */
+
+    @Test
+    public void testFrozenSet()
+    {
+        // pk -> a frozen<set<?>>
+        qt().forAll(TestUtils.cql3Type(bridge))
+            .checkAssert(type ->
+                Tester.builder(TestSchema.builder()
+                                         .withPartitionKey("pk", bridge.uuid())
+                                         .withColumn("a", bridge.set(type).frozen()))
+                      .withExpectedRowCountPerSSTable(Tester.DEFAULT_NUM_ROWS)
+                      .run()
+            );
+    }
+
+    @Test
+    public void testFrozenList()
+    {
+        // pk -> a frozen<list<?>>
+        qt().forAll(TestUtils.cql3Type(bridge))
+            .checkAssert(type ->
+                Tester.builder(TestSchema.builder()
+                                         .withPartitionKey("pk", bridge.uuid())
+                                         .withColumn("a", bridge.list(type).frozen()))
+                      .withExpectedRowCountPerSSTable(Tester.DEFAULT_NUM_ROWS)
+                      .run()
+            );
+    }
+
+    @Test
+    public void testFrozenMap()
+    {
+        // pk -> a frozen<map<?, ?>>
+        qt().withExamples(50)  // Limit number of tests otherwise n x n tests takes too long
+            .forAll(TestUtils.cql3Type(bridge), TestUtils.cql3Type(bridge))
+            .checkAssert((keyType, valueType) ->
+                Tester.builder(TestSchema.builder()
+                                         .withPartitionKey("pk", bridge.uuid())
+                                         .withColumn("a", bridge.map(keyType, valueType).frozen()))
+                      .withExpectedRowCountPerSSTable(Tester.DEFAULT_NUM_ROWS)
+                      .run()
+            );
+    }
+
+    @Test
+    public void testNestedMapSet()
+    {
+        // pk -> a map<text, frozen<set<text>>>
+        Tester.builder(TestSchema.builder()
+                                 .withPartitionKey("pk", bridge.uuid())
+                                 .withColumn("a", bridge.map(bridge.text(), bridge.set(bridge.text()).frozen())))
+              .withNumRandomRows(32)
+              .withExpectedRowCountPerSSTable(32)
+              .run();
+    }
+
+    @Test
+    public void testNestedMapList()
+    {
+        // pk -> a map<text, frozen<list<text>>>
+        Tester.builder(TestSchema.builder()
+                                 .withPartitionKey("pk", bridge.uuid())
+                                 .withColumn("a", bridge.map(bridge.text(), bridge.list(bridge.text()).frozen())))
+              .withNumRandomRows(32)
+              .withExpectedRowCountPerSSTable(32)
+              .run();
+    }
+
+    @Test
+    public void testNestedMapMap()
+    {
+        // pk -> a map<text, frozen<map<bigint, varchar>>>
+        Tester.builder(TestSchema.builder()
+                                 .withPartitionKey("pk", bridge.uuid())
+                                 .withColumn("a", bridge.map(bridge.text(),
+                                                  bridge.map(bridge.bigint(), bridge.varchar()).frozen())))
+              .withNumRandomRows(32)
+              .withExpectedRowCountPerSSTable(32)
+              .dontCheckNumSSTables()
+              .run();
+    }
+
+    @Test
+    public void testFrozenNestedMapMap()
+    {
+        // pk -> a frozen<map<text, <map<int, timestamp>>>
+        Tester.builder(TestSchema.builder()
+                                 .withPartitionKey("pk", bridge.uuid())
+                                 .withColumn("a", bridge.map(bridge.text(),
+                                                             bridge.map(bridge.aInt(), bridge.timestamp())).frozen()))
+              .withNumRandomRows(32)
+              .withExpectedRowCountPerSSTable(32)
+              .dontCheckNumSSTables()
+              .run();
+    }
+
+    /* Filters */
+
+    @Test
+    public void testSinglePartitionKeyFilter()
+    {
+        int numRows = 10;
+        Tester.builder(TestSchema.builder()
+                                 .withPartitionKey("a", bridge.aInt())
+                                 .withColumn("b", bridge.aInt()))
+              .dontWriteRandomData()
+              .withSSTableWriter(writer -> {
+                  for (int row = 0; row < numRows; row++)
+                  {
+                      writer.write(row, row + 1);
+                  }
+              })
+              .withFilter("a=1")
+              .withCheck(dataset -> {
+                  for (Row row : dataset.collectAsList())
+                  {
+                      int a = row.getInt(0);
+                      assertEquals(1, a);
+                  }
+              })
+              .run();
+    }
+
+    @Test
+    public void testMultiplePartitionKeyFilter()
+    {
+        int numRows = 10;
+        int numColumns = 5;
+        Set<String> keys = TestUtils.getKeys(ImmutableList.of(ImmutableList.of("2", "3"),
+                                                              ImmutableList.of("2", "3", "4")));
+        Tester.builder(TestSchema.builder()
+                                 .withPartitionKey("a", bridge.aInt())
+                                 .withPartitionKey("b", bridge.aInt())
+                                 .withColumn("c", bridge.aInt()))
+              .dontWriteRandomData()
+              .withSSTableWriter(writer -> {
+                  for (int row = 0; row < numRows; row++)
+                  {
+                      for (int column = 0; column < numColumns; column++)
+                      {
+                          writer.write(row, (row + 1), column);
+                      }
+                  }
+              })
+              .withFilter("a in (2, 3) and b in (2, 3, 4)")
+              .withCheck(dataset -> {
+                  List<Row> rows = dataset.collectAsList();
+                  assertEquals(2, rows.size());
+                  for (Row row : rows)
+                  {
+                      int a = row.getInt(0);
+                      int b = row.getInt(1);
+                      String key = a + ":" + b;
+                      assertTrue(keys.contains(key));
+                  }
+              })
+              .run();
+    }
+
+    @Test
+    public void testFiltersDoNotMatch()
+    {
+        int numRows = 10;
+        Tester.builder(TestSchema.builder()
+                                 .withPartitionKey("a", bridge.aInt())
+                                 .withColumn("b", bridge.aInt()))
+              .dontWriteRandomData()
+              .withSSTableWriter(writer -> {
+                  for (int row = 0; row < numRows; row++)
+                  {
+                      writer.write(row, row + 1);
+                  }
+              })
+              .withFilter("a=11")
+              .withCheck(dataset -> assertTrue(dataset.collectAsList().isEmpty()))
+              .run();
+    }
+
+    @Test
+    public void testFilterWithClusteringKey()
+    {
+        int numRows = 10;
+        Tester.builder(TestSchema.builder()
+                                 .withPartitionKey("a", bridge.aInt())
+                                 .withClusteringKey("b", bridge.text())
+                                 .withClusteringKey("c", bridge.timestamp()))
+              .dontWriteRandomData()
+              .withSSTableWriter(writer -> {
+                  for (int row = 0; row < numRows; row++)
+                  {
+                      writer.write(200, row < 3 ? "abc" : "def", new java.util.Date(10_000L * (row + 1)));
+                  }
+              })
+              .withFilter("a=200 and b='def'")
+              .withCheck(dataset -> {
+                  List<Row> rows = dataset.collectAsList();
+                  assertFalse(rows.isEmpty());
+                  assertEquals(7, rows.size());
+                  for (Row row : rows)
+                  {
+                      assertEquals(200, row.getInt(0));
+                      assertEquals("def", row.getString(1));
+                  }
+              })
+              .run();
+    }
+
+    @Test
+    public void testUdtNativeTypes()
+    {
+        // pk -> a testudt<b text, c type, d int>
+        qt().forAll(TestUtils.cql3Type(bridge))
+            .checkAssert(type ->
+                Tester.builder(TestSchema.builder()
+                              .withPartitionKey("pk", bridge.uuid())
+                              .withColumn("a", bridge.udt("keyspace", "testudt")
+                                                     .withField("b", bridge.text())
+                                                     .withField("c", type)
+                                                     .withField("d", bridge.aInt())
+                                                     .build()))
+                      .run()
+            );
+    }
+
+    @Test
+    public void testUdtInnerSet()
+    {
+        // pk -> a testudt<b text, c frozen<type>, d int>
+        qt().forAll(TestUtils.cql3Type(bridge))
+            .checkAssert(type ->
+                Tester.builder(TestSchema.builder()
+                                         .withPartitionKey("pk", bridge.uuid())
+                                         .withColumn("a", bridge.udt("keyspace", "testudt")
+                                                                .withField("b", bridge.text())
+                                                                .withField("c", bridge.set(type).frozen())
+                                                                .withField("d", bridge.aInt())
+                                                                .build()))
+                      .run()
+            );
+    }
+
+    @Test
+    public void testUdtInnerList()
+    {
+        // pk -> a testudt<b bigint, c frozen<list<type>>, d boolean>
+        qt().forAll(TestUtils.cql3Type(bridge))
+            .checkAssert(type ->
+                Tester.builder(TestSchema.builder()
+                                         .withPartitionKey("pk", bridge.uuid())
+                                         .withColumn("a", bridge.udt("keyspace", "testudt")
+                                                                .withField("b", bridge.bigint())
+                                                                .withField("c", bridge.list(type).frozen())
+                                                                .withField("d", bridge.bool())
+                                                                .build()))
+                      .run()
+            );
+    }
+
+    @Test
+    public void testUdtInnerMap()
+    {
+        // pk -> a testudt<b float, c frozen<set<uuid>>, d frozen<map<type1, type2>>, e boolean>
+        qt().withExamples(50)
+            .forAll(TestUtils.cql3Type(bridge), TestUtils.cql3Type(bridge))
+            .checkAssert((type1, type2) ->
+                Tester.builder(TestSchema.builder()
+                                         .withPartitionKey("pk", bridge.uuid())
+                                         .withColumn("a", bridge.udt("keyspace", "testudt")
+                                                                .withField("b", bridge.aFloat())
+                                                                .withField("c", bridge.set(bridge.uuid()).frozen())
+                                                                .withField("d", bridge.map(type1, type2).frozen())
+                                                                .withField("e", bridge.bool())
+                                                                .build()))
+                      .run()
+            );
+    }
+
+    @Test
+    public void testMultipleUdts()
+    {
+        // pk -> col1 udt1<a float, b frozen<set<uuid>>, c frozen<set<type>>, d boolean>,
+        //       col2 udt2<a text, b bigint, g varchar>, col3 udt3<int, type, ascii>
+        qt().forAll(TestUtils.cql3Type(bridge))
+            .checkAssert(type ->
+                Tester.builder(TestSchema.builder()
+                                         .withPartitionKey("pk", bridge.uuid())
+                                         .withColumn("col1", bridge.udt("keyspace", "udt1")
+                                                                   .withField("a", bridge.aFloat())
+                                                                   .withField("b", bridge.set(bridge.uuid()).frozen())
+                                                                   .withField("c", bridge.set(type).frozen())
+                                                                   .withField("d", bridge.bool())
+                                                                   .build())
+                                         .withColumn("col2", bridge.udt("keyspace", "udt2")
+                                                                   .withField("a", bridge.text())
+                                                                   .withField("b", bridge.bigint())
+                                                                   .withField("g", bridge.varchar())
+                                                                   .build())
+                                         .withColumn("col3", bridge.udt("keyspace", "udt3")
+                                                                   .withField("a", bridge.aInt())
+                                                                   .withField("b", bridge.list(type).frozen())
+                                                                   .withField("c", bridge.ascii())
+                                                                   .build()))
+                      .run()
+            );
+    }
+
+    @Test
+    public void testNestedUdt()
+    {
+        // pk -> a test_udt<b float, c frozen<set<uuid>>, d frozen<nested_udt<x int, y type, z int>>, e boolean>
+        qt().forAll(TestUtils.cql3Type(bridge))
+            .checkAssert(type ->
+                Tester.builder(TestSchema.builder()
+                                         .withPartitionKey("pk", bridge.uuid())
+                                         .withColumn("a", bridge.udt("keyspace", "test_udt")
+                                                                .withField("b", bridge.aFloat())
+                                                                .withField("c", bridge.set(bridge.uuid()).frozen())
+                                                                .withField("d", bridge.udt("keyspace", "nested_udt")
+                                                                                      .withField("x", bridge.aInt())
+                                                                                      .withField("y", type)
+                                                                                      .withField("z", bridge.aInt())
+                                                                                      .build().frozen())
+                                                                .withField("e", bridge.bool())
+                                                                .build()))
+                      .run()
+            );
+    }
+
+    /* Tuples */
+
+    @Test
+    public void testBasicTuple()
+    {
+        // pk -> a tuple<int, type1, bigint, type2>
+        qt().withExamples(10)
+            .forAll(TestUtils.cql3Type(bridge), TestUtils.cql3Type(bridge))
+            .checkAssert((type1, type2) ->
+                Tester.builder(TestSchema.builder()
+                                         .withPartitionKey("pk", bridge.uuid())
+                                         .withColumn("a", bridge.tuple(bridge.aInt(), type1, bridge.bigint(), type2)))
+                      .run()
+            );
+    }
+
+    @Test
+    public void testTupleWithClusteringKey()
+    {
+        // pk -> col1 type1 -> a tuple<int, type2, bigint>
+        qt().withExamples(10)
+            .forAll(TestUtils.cql3Type(bridge), TestUtils.cql3Type(bridge))
+            .checkAssert((type1, type2) ->
+                Tester.builder(TestSchema.builder()
+                                         .withPartitionKey("pk", bridge.uuid())
+                                         .withClusteringKey("col1", type1)
+                                         .withColumn("a", bridge.tuple(bridge.aInt(), type2, bridge.bigint())))
+                      .run()
+            );
+    }
+
+    @Test
+    public void testNestedTuples()
+    {
+        // pk -> a tuple<varchar, tuple<int, type1, float, varchar, tuple<bigint, boolean, type2>>, timeuuid>
+        // Test tuples nested within tuple
+        qt().withExamples(10)
+            .forAll(TestUtils.cql3Type(bridge), TestUtils.cql3Type(bridge))
+            .checkAssert((type1, type2) ->
+                Tester.builder(TestSchema.builder()
+                                         .withPartitionKey("pk", bridge.uuid())
+                                         .withColumn("a", bridge.tuple(bridge.varchar(),
+                                                                       bridge.tuple(bridge.aInt(),
+                                                                                    type1,
+                                                                                    bridge.aFloat(),
+                                                                                    bridge.varchar(),
+                                                                                    bridge.tuple(bridge.bigint(),
+                                                                                                 bridge.bool(),
+                                                                                                 type2)),
+                                                                       bridge.timeuuid())))
+                      .run()
+            );
+    }
+
+    @Test
+    public void testTupleSet()
+    {
+        // pk -> a tuple<varchar, tuple<int, varchar, float, varchar, set<type>>, timeuuid>
+        // Test set nested within tuple
+        qt().withExamples(10)
+            .forAll(TestUtils.cql3Type(bridge))
+            .checkAssert(type ->
+                Tester.builder(TestSchema.builder()
+                                         .withPartitionKey("pk", bridge.uuid())
+                                         .withColumn("a", bridge.tuple(bridge.varchar(),
+                                                                       bridge.tuple(bridge.aInt(),
+                                                                                    bridge.varchar(),
+                                                                                    bridge.aFloat(),
+                                                                                    bridge.varchar(),
+                                                                                    bridge.set(type)),
+                                                                       bridge.timeuuid())))
+                      .run()
+            );
+    }
+
+    @Test
+    public void testTupleList()
+    {
+        // pk -> a tuple<varchar, tuple<int, varchar, float, varchar, list<type>>, timeuuid>
+        // Test list nested within tuple
+        qt().withExamples(10)
+            .forAll(TestUtils.cql3Type(bridge))
+            .checkAssert(type ->
+                Tester.builder(TestSchema.builder()
+                                         .withPartitionKey("pk", bridge.uuid())
+                                         .withColumn("a", bridge.tuple(bridge.varchar(),
+                                                                       bridge.tuple(bridge.aInt(),
+                                                                                    bridge.varchar(),
+                                                                                    bridge.aFloat(),
+                                                                                    bridge.varchar(),
+                                                                                    bridge.list(type)),
+                                                                       bridge.timeuuid())))
+                      .run()
+            );
+    }
+
+    @Test
+    public void testTupleMap()
+    {
+        // pk -> a tuple<varchar, tuple<int, varchar, float, varchar, map<type1, type2>>, timeuuid>
+        // Test map nested within tuple
+        qt().withExamples(10)
+            .forAll(TestUtils.cql3Type(bridge), TestUtils.cql3Type(bridge))
+            .checkAssert((type1, type2) ->
+                Tester.builder(TestSchema.builder()
+                                        .withPartitionKey("pk", bridge.uuid())
+                                        .withColumn("a", bridge.tuple(bridge.varchar(),
+                                                                      bridge.tuple(bridge.aInt(),
+                                                                                   bridge.varchar(),
+                                                                                   bridge.aFloat(),
+                                                                                   bridge.varchar(),
+                                                                                   bridge.map(type1, type2)),
+                                                                                   bridge.timeuuid())))
+                      .run()
+            );
+    }
+
+    @Test
+    public void testMapTuple()
+    {
+        // pk -> a map<timeuuid, frozen<tuple<boolean, type, timestamp>>>
+        // Test tuple nested within map
+        qt().withExamples(10)
+            .forAll(TestUtils.cql3Type(bridge))
+            .checkAssert(type ->
+                Tester.builder(TestSchema.builder()
+                                         .withPartitionKey("pk", bridge.uuid())
+                                         .withColumn("a", bridge.map(bridge.timeuuid(),
+                                                                     bridge.tuple(bridge.bool(),
+                                                                                  type,
+                                                                                  bridge.timestamp()).frozen())))
+                      .run()
+            );
+    }
+
+    @Test
+    public void testSetTuple()
+    {
+        // pk -> a set<frozen<tuple<type, float, text>>>
+        // Test tuple nested within set
+        qt().withExamples(10)
+            .forAll(TestUtils.cql3Type(bridge))
+            .checkAssert(type ->
+                Tester.builder(TestSchema.builder()
+                                         .withPartitionKey("pk", bridge.uuid())
+                                         .withColumn("a", bridge.set(bridge.tuple(type,
+                                                                                  bridge.aFloat(),
+                                                                                  bridge.text()).frozen())))
+                      .run()
+            );
+    }
+
+    @Test
+    public void testListTuple()
+    {
+        // pk -> a list<frozen<tuple<int, inet, decimal, type>>>
+        // Test tuple nested within map
+        qt().withExamples(10)
+            .forAll(TestUtils.cql3Type(bridge))
+            .checkAssert(type ->
+                Tester.builder(TestSchema.builder()
+                                         .withPartitionKey("pk", bridge.uuid())
+                                         .withColumn("a", bridge.list(bridge.tuple(bridge.aInt(),
+                                                                                   bridge.inet(),
+                                                                                   bridge.decimal(),
+                                                                                   type).frozen())))
+                      .run()
+            );
+    }
+
+    @Test
+    public void testTupleUDT()
+    {
+        // pk -> a tuple<varchar, frozen<nested_udt<x int, y type, z int>>, timeuuid>
+        // Test tuple with inner UDT
+        qt().withExamples(10)
+            .forAll(TestUtils.cql3Type(bridge))
+            .checkAssert(type ->
+                Tester.builder(TestSchema.builder()
+                                         .withPartitionKey("pk", bridge.uuid())
+                                         .withColumn("a", bridge.tuple(bridge.varchar(),
+                                                                       bridge.udt("keyspace", "nested_udt")
+                                                                             .withField("x", bridge.aInt())
+                                                                             .withField("y", type)
+                                                                             .withField("z", bridge.aInt())
+                                                                             .build().frozen(),
+                                                                       bridge.timeuuid())))
+                      .run()
+            );
+    }
+
+    @Test
+    public void testUDTTuple()
+    {
+        // pk -> a nested_udt<x text, y tuple<int, float, type, timestamp>, z ascii>
+        // Test UDT with inner tuple
+        qt().withExamples(10)
+            .forAll(TestUtils.cql3Type(bridge))
+            .checkAssert(type ->
+                Tester.builder(TestSchema.builder()
+                                         .withPartitionKey("pk", bridge.uuid())
+                                         .withColumn("a", bridge.udt("keyspace", "nested_udt")
+                                                                .withField("x", bridge.text())
+                                                                .withField("y", bridge.tuple(bridge.aInt(),
+                                                                                             bridge.aFloat(),
+                                                                                             type,
+                                                                                             bridge.timestamp()))
+                                                                .withField("z", bridge.ascii())
+                                                                .build()))
+                      .run()
+            );
+    }
+
+    @Test
+    public void testTupleClusteringKey()
+    {
+        qt().forAll(TestUtils.cql3Type(bridge))
+            .checkAssert(type ->
+                Tester.builder(TestSchema.builder()
+                                         .withPartitionKey("pk", bridge.uuid())
+                                         .withClusteringKey("ck", bridge.tuple(bridge.aInt(),
+                                                                               bridge.text(),
+                                                                               type,
+                                                                               bridge.aFloat()))
+                                         .withColumn("a", bridge.text())
+                                         .withColumn("b", bridge.aInt())
+                                         .withColumn("c", bridge.ascii()))
+                      .run()
+            );
+    }
+
+    @Test
+    public void testUdtClusteringKey()
+    {
+        qt().forAll(TestUtils.cql3Type(bridge))
+            .checkAssert(type ->
+                Tester.builder(TestSchema.builder()
+                                         .withPartitionKey("pk", bridge.uuid())
+                                         .withClusteringKey("ck", bridge.udt("keyspace", "udt1")
+                                                                        .withField("a", bridge.text())
+                                                                        .withField("b", type)
+                                                                        .withField("c", bridge.aInt())
+                                                                        .build().frozen())
+                                         .withColumn("a", bridge.text())
+                                         .withColumn("b", bridge.aInt())
+                                         .withColumn("c", bridge.ascii()))
+                      .run()
+            );
+    }
+
+    @Test
+    public void testComplexSchema()
+    {
+        String keyspace = "complex_schema2";
+        CqlField.CqlUdt udt1 = bridge.udt(keyspace, "udt1")
+                                     .withField("time", bridge.bigint())
+                                     .withField("\"time_offset_minutes\"", bridge.aInt())
+                                     .build();
+        CqlField.CqlUdt udt2 = bridge.udt(keyspace, "udt2")
+                                     .withField("\"version\"", bridge.text())
+                                     .withField("\"id\"", bridge.text())
+                                     .withField("platform", bridge.text())
+                                     .withField("time_range", bridge.text())
+                                     .build();
+        CqlField.CqlUdt udt3 = bridge.udt(keyspace, "udt3")
+                                     .withField("field", bridge.text())
+                                     .withField("\"time_with_zone\"", udt1)
+                                     .build();
+        CqlField.CqlUdt udt4 = bridge.udt(keyspace, "udt4")
+                                     .withField("\"first_seen\"", udt3.frozen())
+                                     .withField("\"last_seen\"", udt3.frozen())
+                                     .withField("\"first_transaction\"", udt3.frozen())
+                                     .withField("\"last_transaction\"", udt3.frozen())
+                                     .withField("\"first_listening\"", udt3.frozen())
+                                     .withField("\"last_listening\"", udt3.frozen())
+                                     .withField("\"first_reading\"", udt3.frozen())
+                                     .withField("\"last_reading\"", udt3.frozen())
+                                     .withField("\"output_event\"", bridge.text())
+                                     .withField("\"event_history\"", bridge.map(bridge.bigint(),
+                                                                                bridge.map(bridge.text(),
+                                                                                           bridge.bool()).frozen()
+                                                                               ).frozen())
+                                     .build();
+
+        Tester.builder(keyspace1 -> TestSchema.builder()
+                                              .withKeyspace(keyspace1)
+                                              .withPartitionKey("\"consumerId\"", bridge.text())
+                                              .withClusteringKey("dimensions", udt2.frozen())
+                                              .withColumn("fields", udt4.frozen())
+                                              .withColumn("first_transition_time", udt1.frozen())
+                                              .withColumn("last_transition_time", udt1.frozen())
+                                              .withColumn("prev_state_id", bridge.text())
+                                              .withColumn("state_id", bridge.text()))
+              .run();
+    }
+
+    @Test
+    public void testNestedFrozenUDT()
+    {
+        // "(a bigint PRIMARY KEY, b <map<int, frozen<testudt>>>)"
+        // testudt(a text, b bigint, c int)
+        CqlField.CqlUdt testudt = bridge.udt("nested_frozen_udt", "testudt")
+                                        .withField("a", bridge.text())
+                                        .withField("b", bridge.bigint())
+                                        .withField("c", bridge.aInt())
+                                        .build();
+        Tester.builder(keyspace -> TestSchema.builder()
+                                             .withKeyspace(keyspace)
+                                             .withPartitionKey("a", bridge.bigint())
+                                             .withColumn("b", bridge.map(bridge.aInt(), testudt.frozen())))
+              .run();
+    }
+
+    @Test
+    public void testDeepNestedUDT()
+    {
+        String keyspace = "deep_nested_frozen_udt";
+        CqlField.CqlUdt udt1 = bridge.udt(keyspace, "udt1")
+                                     .withField("a", bridge.text())
+                                     .withField("b", bridge.aInt())
+                                     .withField("c", bridge.bigint())
+                                     .build();
+        CqlField.CqlUdt udt2 = bridge.udt(keyspace, "udt2")
+                                     .withField("a", bridge.aInt())
+                                     .withField("b", bridge.set(bridge.uuid()))
+                                     .build();
+        CqlField.CqlUdt udt3 = bridge.udt(keyspace, "udt3")
+                                     .withField("a", bridge.aInt())
+                                     .withField("b", bridge.set(bridge.uuid()))
+                                     .build();
+        CqlField.CqlUdt udt4 = bridge.udt(keyspace, "udt4")
+                                     .withField("a", bridge.text())
+                                     .withField("b", bridge.text())
+                                     .withField("c", bridge.uuid())
+                                     .withField("d", bridge.list(bridge.tuple(bridge.text(),
+                                                                              bridge.bigint()).frozen()
+                                                                             ).frozen())
+                                     .build();
+        CqlField.CqlUdt udt5 = bridge.udt(keyspace, "udt5")
+                                     .withField("a", bridge.text())
+                                     .withField("b", bridge.text())
+                                     .withField("c", bridge.bigint())
+                                     .withField("d", bridge.set(udt4.frozen()))
+                                     .build();
+        CqlField.CqlUdt udt6 = bridge.udt(keyspace, "udt6")
+                                     .withField("a", bridge.text())
+                                     .withField("b", bridge.text())
+                                     .withField("c", bridge.aInt())
+                                     .withField("d", bridge.aInt())
+                                     .build();
+        CqlField.CqlUdt udt7 = bridge.udt(keyspace, "udt7")
+                                     .withField("a", bridge.text())
+                                     .withField("b", bridge.uuid())
+                                     .withField("c", bridge.bool())
+                                     .withField("d", bridge.bool())
+                                     .withField("e", bridge.bool())
+                                     .withField("f", bridge.bigint())
+                                     .withField("g", bridge.bigint())
+                                     .build();
+
+        CqlField.CqlUdt udt8 = bridge.udt(keyspace, "udt8")
+                                     .withField("a", bridge.text())
+                                     .withField("b", bridge.bool())
+                                     .withField("c", bridge.bool())
+                                     .withField("d", bridge.bool())
+                                     .withField("e", bridge.bigint())
+                                     .withField("f", bridge.bigint())
+                                     .withField("g", bridge.uuid())
+                                     .withField("h", bridge.bigint())
+                                     .withField("i", bridge.uuid())
+                                     .withField("j", bridge.uuid())
+                                     .withField("k", bridge.uuid())
+                                     .withField("l", bridge.uuid())
+                                     .withField("m", bridge.aInt())
+                                     .withField("n", bridge.timestamp())
+                                     .withField("o", bridge.text())
+                                     .build();
+
+        Tester.builder(keyspace1 -> TestSchema.builder()
+                                              .withKeyspace(keyspace1)
+                                              .withPartitionKey("pk", bridge.uuid())
+                                              .withClusteringKey("ck", bridge.uuid())
+                                              .withColumn("a", udt3.frozen())
+                                              .withColumn("b", udt2.frozen())
+                                              .withColumn("c", bridge.set(bridge.tuple(udt1,
+                                                                                       bridge.text()).frozen()))
+                                              .withColumn("d", bridge.set(bridge.tuple(bridge.bigint(),
+                                                                                       bridge.text()).frozen()))
+                                              .withColumn("e", bridge.set(bridge.tuple(udt2,
+                                                                                       bridge.text()).frozen()))
+                                              .withColumn("f", bridge.set(udt7.frozen()))
+                                              .withColumn("g", bridge.map(bridge.aInt(),
+                                                                          bridge.set(bridge.text()).frozen()))
+                                              .withColumn("h", bridge.set(bridge.tinyint()))
+                                              .withColumn("i", bridge.map(bridge.text(),
+                                                                          udt6.frozen()))
+                                              .withColumn("j", bridge.map(bridge.text(),
+                                                                          bridge.map(bridge.text(),
+                                                                                     bridge.text()).frozen()))
+                                              .withColumn("k", bridge.list(bridge.tuple(bridge.text(),
+                                                                                        bridge.text(),
+                                                                                        bridge.text()).frozen()))
+                                              .withColumn("l", bridge.list(udt5.frozen()))
+                                              .withColumn("m", udt8.frozen())
+                                              .withMinCollectionSize(4))
+              .withNumRandomRows(50)
+              .withNumRandomSSTables(2)
+              .run();
+    }
+
+    /* BigDecimal/Integer Tests */
+
+    @Test
+    public void testBigDecimal()
+    {
+        Tester.builder(TestSchema.builder()
+                                 .withPartitionKey("pk", bridge.uuid())
+                                 .withColumn("c1", bridge.decimal())
+                                 .withColumn("c2", bridge.text()))
+              .run();
+    }
+
+    @Test
+    public void testBigInteger()
+    {
+        Tester.builder(TestSchema.builder()
+                                 .withPartitionKey("pk", bridge.uuid())
+                                 .withColumn("c1", bridge.varint())
+                                 .withColumn("c2", bridge.text()))
+              .run();
+    }
+
+    @Test
+    public void testUdtFieldOrdering()
+    {
+        String keyspace = "udt_field_ordering";
+        CqlField.CqlUdt udt1 = bridge.udt(keyspace, "udt1")
+                                     .withField("c", bridge.text())
+                                     .withField("b", bridge.uuid())
+                                     .withField("a", bridge.bool())
+                                     .build();
+        Tester.builder(keyspace1 -> TestSchema.builder()
+                                              .withKeyspace(keyspace1)
+                                              .withPartitionKey("pk", bridge.uuid())
+                                              .withColumn("a", bridge.set(udt1.frozen())))
+              .run();
+    }
+
+    @Test
+    @SuppressWarnings("unchecked")
+    public void testUdtTupleInnerNulls()
+    {
+        CqlField.CqlUdt udtType = bridge.udt("udt_inner_nulls", "udt")
+                                        .withField("a", bridge.uuid())
+                                        .withField("b", bridge.text())
+                                        .build();
+        CqlField.CqlTuple tupleType = bridge.tuple(bridge.bigint(), bridge.text(), bridge.aInt());
+
+        int numRows = 50;
+        int midPoint = numRows / 2;
+        Map<UUID, Set<Map<String, Object>>> udtSetValues = new LinkedHashMap<>(numRows);
+        Map<UUID, Object[]> tupleValues = new LinkedHashMap<>(numRows);
+        for (int tupleIndex = 0; tupleIndex < numRows; tupleIndex++)
+        {
+            UUID pk = UUID.randomUUID();
+            Set<Map<String, Object>> udtSet = new HashSet<>(numRows);
+            for (int udtIndex = 0; udtIndex < numRows; udtIndex++)
+            {
+                Map<String, Object> udt = Maps.newHashMapWithExpectedSize(2);
+                udt.put("a", UUID.randomUUID());
+                udt.put("b", udtIndex < midPoint ? UUID.randomUUID().toString() : null);
+                udtSet.add(udt);
+            }
+            Object[] tuple = new Object[]{RandomUtils.RANDOM.nextLong(),
+                                          tupleIndex < midPoint ? UUID.randomUUID().toString() : null,
+                                          RandomUtils.RANDOM.nextInt()};
+
+            udtSetValues.put(pk, udtSet);
+            tupleValues.put(pk, tuple);
+        }
+
+        Tester.builder(keyspace -> TestSchema.builder()
+                                             .withKeyspace(keyspace)
+                                             .withPartitionKey("pk", bridge.uuid())
+                                             .withColumn("a", bridge.set(udtType.frozen()))
+                                             .withColumn("b", tupleType))
+              .dontWriteRandomData()
+              .withSSTableWriter(writer -> {
+                  for (UUID pk : udtSetValues.keySet())
+                  {
+                      Set<Object> udtSet = udtSetValues.get(pk).stream()
+                                                               .map(map ->  bridge.toUserTypeValue(udtType, map))
+                                                               .collect(Collectors.toSet());
+                      Object tuple = bridge.toTupleValue(tupleType, tupleValues.get(pk));
+
+                      writer.write(pk, udtSet, tuple);
+                  }
+              })
+              .withCheck(dataset -> {
+                  for (Row row : dataset.collectAsList())
+                  {
+                      UUID pk = UUID.fromString(row.getString(0));
+
+                      Set<Map<String, Object>> expectedUdtSet = udtSetValues.get(pk);
+                      List<Row> udtSet = mutableSeqAsJavaList((AbstractSeq<Row>) row.get(1));
+                      assertEquals(expectedUdtSet.size(), udtSet.size());
+                      for (Row udt : udtSet)
+                      {
+                          Map<String, Object> expectedUdt = Maps.newHashMapWithExpectedSize(2);
+                          expectedUdt.put("a", UUID.fromString(udt.getString(0)));
+                          expectedUdt.put("b", udt.getString(1));
+                          assertTrue(expectedUdtSet.contains(expectedUdt));
+                      }
+
+                      Object[] expectedTuple = tupleValues.get(pk);
+                      Row tuple = (Row) row.get(2);
+                      assertEquals(expectedTuple.length, tuple.length());
+                      assertEquals(expectedTuple[0], tuple.getLong(0));
+                      assertEquals(expectedTuple[1], tuple.getString(1));
+                      assertEquals(expectedTuple[2], tuple.getInt(2));
+                  }
+              })
+              .run();
+    }
+
+    /* Complex Clustering Keys */
+
+    @Test
+    public void testUdtsWithNulls()
+    {
+        CqlField.CqlUdt type = bridge.udt("udt_with_nulls", "udt1")
+                                     .withField("a", bridge.text())
+                                     .withField("b", bridge.text())
+                                     .withField("c", bridge.text())
+                                     .build();
+        Map<Long, Map<String, Object>> values = new HashMap<>(Tester.DEFAULT_NUM_ROWS);
+
+        Tester.builder(keyspace -> TestSchema.builder()
+                                             .withKeyspace(keyspace)
+                                             .withPartitionKey("pk", bridge.bigint())
+                                             .withClusteringKey("ck", type.frozen())
+                                             .withColumn("col1", bridge.text())
+                                             .withColumn("col2", bridge.timestamp())
+                                             .withColumn("col3", bridge.aInt()))
+              .dontWriteRandomData()
+              .withSSTableWriter(writer -> {
+                  int midPoint = Tester.DEFAULT_NUM_ROWS / 2;
+                  for (long pk = 0; pk < Tester.DEFAULT_NUM_ROWS; pk++)
+                  {
+                      Map<String, Object> value = ImmutableMap.of(
+                            pk < midPoint ? "a" : "b", RandomUtils.randomValue(bridge.text()).toString(),
+                            "c", RandomUtils.randomValue(bridge.text()).toString());
+                      values.put(pk, value);
+                      writer.write(pk, bridge.toUserTypeValue(type, value),
+                                       RandomUtils.randomValue(bridge.text()),
+                                       RandomUtils.randomValue(bridge.timestamp()),
+                                       RandomUtils.randomValue(bridge.aInt()));
+                  }
+              })
+              .withCheck(dataset -> {
+                  Map<Long, Row> rows = dataset.collectAsList().stream()
+                                                               .collect(Collectors.toMap(row -> row.getLong(0),
+                                                                                         row -> row.getStruct(1)));
+                  assertEquals(values.size(), rows.size());
+                  for (Map.Entry<Long, Row> pk : rows.entrySet())
+                  {
+                      assertEquals(values.get(pk.getKey()).get("a"), pk.getValue().getString(0));
+                      assertEquals(values.get(pk.getKey()).get("b"), pk.getValue().getString(1));
+                      assertEquals(values.get(pk.getKey()).get("c"), pk.getValue().getString(2));
+                  }
+              })
+              .run();
+    }
+
+    @Test
+    public void testMapClusteringKey()
+    {
+        Tester.builder(TestSchema.builder()
+                                 .withPartitionKey("pk", bridge.uuid())
+                                 .withClusteringKey("ck", bridge.map(bridge.bigint(), bridge.text()).frozen())
+                                 .withColumn("c1", bridge.text())
+                                 .withColumn("c2", bridge.text())
+                                 .withColumn("c3", bridge.text()))
+              .withNumRandomRows(5)
+              .run();
+    }
+
+    @Test
+    public void testListClusteringKey()
+    {
+        Tester.builder(TestSchema.builder()
+                                 .withPartitionKey("pk", bridge.uuid())
+                                 .withClusteringKey("ck", bridge.list(bridge.bigint()).frozen())
+                                 .withColumn("c1", bridge.text())
+                                 .withColumn("c2", bridge.text())
+                                 .withColumn("c3", bridge.text()))
+              .run();
+    }
+
+    @Test
+    public void testSetClusteringKey()
+    {
+        Tester.builder(TestSchema.builder()
+                                 .withPartitionKey("pk", bridge.uuid())
+                                 .withClusteringKey("ck", bridge.set(bridge.aFloat()).frozen())
+                                 .withColumn("c1", bridge.text())
+                                 .withColumn("c2", bridge.text())
+                                 .withColumn("c3", bridge.text()))
+              .run();
+    }
+
+    @Test
+    public void testUdTClusteringKey()
+    {
+        Tester.builder(keyspace -> TestSchema.builder()
+                                             .withKeyspace(keyspace)
+                                             .withPartitionKey("pk", bridge.uuid())
+                                             .withClusteringKey("ck", bridge.udt("udt_clustering_key", "udt1")
+                                                                            .withField("a", bridge.text())
+                                                                            .withField("b", bridge.aFloat())
+                                                                            .withField("c", bridge.bigint())
+                                                                            .build().frozen())
+                                             .withColumn("c1", bridge.text())
+                                             .withColumn("c2", bridge.text())
+                                             .withColumn("c3", bridge.text()))
+              .run();
+    }
+
+    /* Column Prune Filters */
+
+    // CHECKSTYLE IGNORE: Despite being static and final, this is a mutable field not to be confused with a constant
+    private static final AtomicLong skippedRawBytes = new AtomicLong(0L);
+    private static final AtomicLong skippedInputStreamBytes = new AtomicLong(0L);  // CHECKSTYLE IGNORE: Ditto
+    private static final AtomicLong skippedRangeBytes = new AtomicLong(0L);        // CHECKSTYLE IGNORE: Ditto
+
+    private static void resetStats()
+    {
+        skippedRawBytes.set(0L);
+        skippedInputStreamBytes.set(0L);
+        skippedRangeBytes.set(0L);
+    }
+
+    @SuppressWarnings("unused")  // Actually used via reflection in testLargeBlobExclude()
+    public static final Stats STATS = new Stats()
+    {
+        @Override
+        public void skippedBytes(long length)
+        {
+            skippedRawBytes.addAndGet(length);
+        }
+
+        @Override
+        public void inputStreamBytesSkipped(SSTableSource<? extends SSTable> ssTable,
+                                            long bufferedSkipped,
+                                            long rangeSkipped)
+        {
+            skippedInputStreamBytes.addAndGet(bufferedSkipped);
+            skippedRangeBytes.addAndGet(rangeSkipped);
+        }
+    };
+
+    @Test
+    public void testLargeBlobExclude()
+    {
+        qt().forAll(booleans().all())
+            .checkAssert(enableCompression ->
+                Tester.builder(TestSchema.builder()
+                                         .withPartitionKey("pk", bridge.uuid())
+                                         .withClusteringKey("ck", bridge.aInt())
+                                         .withColumn("a", bridge.bigint())
+                                         .withColumn("b", bridge.text())
+                                         .withColumn("c", bridge.blob())
+                                         .withBlobSize(400000)  // Override blob size to write large blobs that we can skip
+                                         .withCompression(enableCompression))
+                      // Test with LZ4 enabled & disabled
+                      .withColumns("pk", "ck", "a")  // Partition/clustering keys are always required
+                      .withExpectedRowCountPerSSTable(Tester.DEFAULT_NUM_ROWS)
+                      .withStatsClass(EndToEndTests.class.getName() + ".STATS")  // Override stats so we can count bytes skipped
+                      .withCheck(dataset -> {
+                          EndToEndTests.resetStats();
+                          List<Row> rows = dataset.collectAsList();
+                          assertFalse(rows.isEmpty());
+                          for (Row row : rows)
+                          {
+                              assertTrue(row.schema().getFieldIndex("pk").isDefined());
+                              assertTrue(row.schema().getFieldIndex("ck").isDefined());
+                              assertTrue(row.schema().getFieldIndex("a").isDefined());
+                              assertFalse(row.schema().getFieldIndex("b").isDefined());
+                              assertFalse(row.schema().getFieldIndex("c").isDefined());
+                              assertEquals(3, row.length());
+                              assertTrue(row.get(0) instanceof String);
+                              assertTrue(row.get(1) instanceof Integer);
+                              assertTrue(row.get(2) instanceof Long);
+                          }
+                          assertTrue(skippedRawBytes.get() > 50_000_000);
+                          assertTrue(skippedInputStreamBytes.get() > 2_500_000);
+                          assertTrue(skippedRangeBytes.get() > 5_000_000);
+                      })
+                      .withReset(EndToEndTests::resetStats)
+                      .run()
+            );
+    }
+
+    @Test
+    public void testExcludeColumns()
+    {
+        Tester.builder(TestSchema.builder()
+                                 .withPartitionKey("pk", bridge.uuid())
+                                 .withClusteringKey("ck", bridge.aInt())
+                                 .withColumn("a", bridge.bigint())
+                                 .withColumn("b", bridge.text())
+                                 .withColumn("c", bridge.ascii())
+                                 .withColumn("d", bridge.list(bridge.text()))
+                                 .withColumn("e", bridge.map(bridge.bigint(), bridge.text())))
+              .withColumns("pk", "ck", "a", "c", "e")
+              .withExpectedRowCountPerSSTable(Tester.DEFAULT_NUM_ROWS)
+              .withCheck(dataset -> {
+                  List<Row> rows = dataset.collectAsList();
+                  assertFalse(rows.isEmpty());
+                  for (Row row : rows)
+                  {
+                      assertTrue(row.schema().getFieldIndex("pk").isDefined());
+                      assertTrue(row.schema().getFieldIndex("ck").isDefined());
+                      assertTrue(row.schema().getFieldIndex("a").isDefined());
+                      assertFalse(row.schema().getFieldIndex("b").isDefined());
+                      assertTrue(row.schema().getFieldIndex("c").isDefined());
+                      assertFalse(row.schema().getFieldIndex("d").isDefined());
+                      assertTrue(row.schema().getFieldIndex("e").isDefined());
+                      assertEquals(5, row.length());
+                      assertTrue(row.get(0) instanceof String);
+                      assertTrue(row.get(1) instanceof Integer);
+                      assertTrue(row.get(2) instanceof Long);
+                      assertTrue(row.get(3) instanceof String);
+                      assertTrue(row.get(4) instanceof scala.collection.immutable.Map);
+                  }
+              })
+              .run();
+    }
+
+    @Test
+    public void testUpsertExcludeColumns()
+    {
+        Tester.builder(TestSchema.builder()
+                                 .withPartitionKey("pk", bridge.uuid())
+                                 .withClusteringKey("ck", bridge.aInt())
+                                 .withColumn("a", bridge.bigint())
+                                 .withColumn("b", bridge.text())
+                                 .withColumn("c", bridge.ascii())
+                                 .withColumn("d", bridge.list(bridge.text()))
+                                 .withColumn("e", bridge.map(bridge.bigint(), bridge.text())))
+              .withColumns("pk", "ck", "a", "c", "e")
+              .withUpsert()
+              .withExpectedRowCountPerSSTable(Tester.DEFAULT_NUM_ROWS)
+              .withCheck(dataset -> {
+                  List<Row> rows = dataset.collectAsList();
+                  assertFalse(rows.isEmpty());
+                  for (Row row : rows)
+                  {
+                      assertTrue(row.schema().getFieldIndex("pk").isDefined());
+                      assertTrue(row.schema().getFieldIndex("ck").isDefined());
+                      assertTrue(row.schema().getFieldIndex("a").isDefined());
+                      assertFalse(row.schema().getFieldIndex("b").isDefined());
+                      assertTrue(row.schema().getFieldIndex("c").isDefined());
+                      assertFalse(row.schema().getFieldIndex("d").isDefined());
+                      assertTrue(row.schema().getFieldIndex("e").isDefined());
+                      assertEquals(5, row.length());
+                      assertTrue(row.get(0) instanceof String);
+                      assertTrue(row.get(1) instanceof Integer);
+                      assertTrue(row.get(2) instanceof Long);
+                      assertTrue(row.get(3) instanceof String);
+                      assertTrue(row.get(4) instanceof scala.collection.immutable.Map);
+                  }
+              })
+              .run();
+    }
+
+    @Test
+    public void testExcludeNoColumns()
+    {
+        // Include all columns
+        Tester.builder(TestSchema.builder()
+                                 .withPartitionKey("pk", bridge.uuid())
+                                 .withClusteringKey("ck", bridge.aInt())
+                                 .withColumn("a", bridge.bigint())
+                                 .withColumn("b", bridge.text())
+                                 .withColumn("c", bridge.ascii())
+                                 .withColumn("d", bridge.bigint())
+                                 .withColumn("e", bridge.aFloat())
+                                 .withColumn("f", bridge.bool()))
+              .withColumns("pk", "ck", "a", "b", "c", "d", "e", "f")
+              .withExpectedRowCountPerSSTable(Tester.DEFAULT_NUM_ROWS)
+              .run();
+    }
+
+    @Test
+    public void testUpsertExcludeNoColumns()
+    {
+        // Include all columns
+        Tester.builder(TestSchema.builder()
+                                 .withPartitionKey("pk", bridge.uuid())
+                                 .withClusteringKey("ck", bridge.aInt())
+                                 .withColumn("a", bridge.bigint())
+                                 .withColumn("b", bridge.text())
+                                 .withColumn("c", bridge.ascii())
+                                 .withColumn("d", bridge.bigint())
+                                 .withColumn("e", bridge.aFloat())
+                                 .withColumn("f", bridge.bool()))
+              .withColumns("pk", "ck", "a", "b", "c", "d", "e", "f")
+              .withUpsert()
+              .withExpectedRowCountPerSSTable(Tester.DEFAULT_NUM_ROWS)
+              .run();
+    }
+
+    @Test
+    public void testExcludeAllColumns()
+    {
+        // Exclude all columns except for partition/clustering keys
+        Tester.builder(TestSchema.builder()
+                                 .withPartitionKey("pk", bridge.uuid())
+                                 .withClusteringKey("ck", bridge.aInt())
+                                 .withColumn("a", bridge.bigint())
+                                 .withColumn("b", bridge.text())
+                                 .withColumn("c", bridge.ascii())
+                                 .withColumn("d", bridge.bigint())
+                                 .withColumn("e", bridge.aFloat())
+                                 .withColumn("f", bridge.bool()))
+              .withColumns("pk", "ck")
+              .withExpectedRowCountPerSSTable(Tester.DEFAULT_NUM_ROWS)
+              .run();
+    }
+
+    @Test
+    public void testUpsertExcludeAllColumns()
+    {
+        // Exclude all columns except for partition/clustering keys
+        Tester.builder(TestSchema.builder()
+                                 .withPartitionKey("pk", bridge.uuid())
+                                 .withClusteringKey("ck", bridge.aInt())
+                                 .withColumn("a", bridge.bigint())
+                                 .withColumn("b", bridge.text())
+                                 .withColumn("c", bridge.ascii())
+                                 .withColumn("d", bridge.bigint())
+                                 .withColumn("e", bridge.aFloat())
+                                 .withColumn("f", bridge.bool()))
+              .withUpsert()
+              .withColumns("pk", "ck")
+              .withExpectedRowCountPerSSTable(Tester.DEFAULT_NUM_ROWS)
+              .run();
+    }
+
+    @Test
+    public void testExcludePartitionOnly()
+    {
+        Tester.builder(TestSchema.builder()
+                                 .withPartitionKey("pk", bridge.uuid()))
+              .withColumns("pk")
+              .withExpectedRowCountPerSSTable(Tester.DEFAULT_NUM_ROWS)
+              .run();
+    }
+
+    @Test
+    public void testExcludeKeysOnly()
+    {
+        Tester.builder(TestSchema.builder()
+                                 .withPartitionKey("pk", bridge.uuid())
+                                 .withClusteringKey("ck1", bridge.text())
+                                 .withClusteringKey("ck2", bridge.bigint()))
+              .withColumns("pk", "ck1", "ck2")
+              .withExpectedRowCountPerSSTable(Tester.DEFAULT_NUM_ROWS)
+              .run();
+    }
+
+    @Test
+    public void testExcludeKeysStaticColumnOnly()
+    {
+        Tester.builder(TestSchema.builder()
+                                 .withPartitionKey("pk", bridge.uuid())
+                                 .withClusteringKey("ck1", bridge.text())
+                                 .withClusteringKey("ck2", bridge.bigint())
+                                 .withStaticColumn("c1", bridge.timestamp()))
+              .withColumns("pk", "ck1", "ck2", "c1")
+              .withExpectedRowCountPerSSTable(Tester.DEFAULT_NUM_ROWS)
+              .run();
+    }
+
+    @Test
+    public void testExcludeStaticColumn()
+    {
+        // Exclude static columns
+        Tester.builder(TestSchema.builder()
+                                 .withPartitionKey("pk", bridge.uuid())
+                                 .withClusteringKey("ck", bridge.aInt())
+                                 .withStaticColumn("a", bridge.text())
+                                 .withStaticColumn("b", bridge.timestamp())
+                                 .withColumn("c", bridge.bigint())
+                                 .withStaticColumn("d", bridge.uuid()))
+              .withColumns("pk", "ck", "c")
+              .withExpectedRowCountPerSSTable(Tester.DEFAULT_NUM_ROWS)
+              .run();
+    }
+
+    @Test
+    public void testUpsertExcludeStaticColumn()
+    {
+        // Exclude static columns
+        Tester.builder(TestSchema.builder()
+                                 .withPartitionKey("pk", bridge.uuid())
+                                 .withClusteringKey("ck", bridge.aInt())
+                                 .withStaticColumn("a", bridge.text())
+                                 .withStaticColumn("b", bridge.timestamp())
+                                 .withColumn("c", bridge.bigint())
+                                 .withStaticColumn("d", bridge.uuid()))
+              .withColumns("pk", "ck", "c")
+              .withUpsert()
+              .withExpectedRowCountPerSSTable(Tester.DEFAULT_NUM_ROWS)
+              .run();
+    }
+
+    @Test
+    public void testLastModifiedTimestampAddedWithStaticColumn()
+    {
+        int numRows = 5;
+        int numColumns = 5;
+        long leastExpectedTimestamp = Timestamp.from(Instant.now()).getTime();
+        Set<Pair<Integer, Long>> observedLMT = new HashSet<>();
+        Tester.builder(TestSchema.builder()
+                                 .withPartitionKey("pk", bridge.aInt())
+                                 .withClusteringKey("ck", bridge.aInt())
+                                 .withStaticColumn("a", bridge.text()))
+              .dontWriteRandomData()
+              .withSSTableWriter(writer -> {
+                  for (int row = 0; row < numRows; row++)
+                  {
+                      for (int column = 0; column < numColumns; column++)
+                      {
+                          // Makes sure the insertion time of each row is unique
+                          Uninterruptibles.sleepUninterruptibly(1, TimeUnit.MILLISECONDS);
+                          writer.write(row, column, "text" + column);
+                      }
+                  }
+              })
+              .withLastModifiedTimestampColumn()
+              .withCheck(dataset -> {
+                  for (Row row : dataset.collectAsList())
+                  {
+                      assertEquals(4, row.length());
+                      assertEquals("text4", String.valueOf(row.get(2)));
+                      long lmt = row.getTimestamp(3).getTime();
+                      assertTrue(lmt > leastExpectedTimestamp);
+                      // Due to the static column so the LMT is the same per partition.
+                      // Using the pair of ck and lmt for uniqueness check.
+                      assertTrue("Observed a duplicated LMT", observedLMT.add(Pair.of(row.getInt(1), lmt)));
+                  }
+              })
+              .run();
+    }
+
+    @Test
+    public void testLastModifiedTimestampWithExcludeColumns()
+    {
+        Tester.builder(TestSchema.builder().withPartitionKey("pk", bridge.uuid())
+                                 .withClusteringKey("ck", bridge.aInt())
+                                 .withColumn("a", bridge.bigint())
+                                 .withColumn("b", bridge.text())
+                                 .withColumn("c", bridge.ascii())
+                                 .withColumn("d", bridge.list(bridge.text()))
+                                 .withColumn("e", bridge.map(bridge.bigint(), bridge.text())))
+              .withLastModifiedTimestampColumn()
+              .withColumns("pk", "ck", "a", "c", "e", "last_modified_timestamp")
+              .withExpectedRowCountPerSSTable(Tester.DEFAULT_NUM_ROWS)
+              .withCheck(dataset -> {
+                  List<Row> rows = dataset.collectAsList();
+                  assertFalse(rows.isEmpty());
+                  for (Row row : rows)
+                  {
+                      assertTrue(row.schema().getFieldIndex("pk").isDefined());
+                      assertTrue(row.schema().getFieldIndex("ck").isDefined());
+                      assertTrue(row.schema().getFieldIndex("a").isDefined());
+                      assertFalse(row.schema().getFieldIndex("b").isDefined());
+                      assertTrue(row.schema().getFieldIndex("c").isDefined());
+                      assertFalse(row.schema().getFieldIndex("d").isDefined());
+                      assertTrue(row.schema().getFieldIndex("e").isDefined());
+                      assertTrue(row.schema().getFieldIndex("last_modified_timestamp").isDefined());
+                      assertEquals(6, row.length());
+                      assertTrue(row.get(0) instanceof String);
+                      assertTrue(row.get(1) instanceof Integer);
+                      assertTrue(row.get(2) instanceof Long);
+                      assertTrue(row.get(3) instanceof String);
+                      assertTrue(row.get(4) instanceof scala.collection.immutable.Map);
+                      assertTrue(row.get(5) instanceof java.sql.Timestamp);
+                      assertTrue(((java.sql.Timestamp) row.get(5)).getTime() > 0);
+                  }
+              })
+              .run();
+    }
+
+    @Test
+    public void testLastModifiedTimestampAddedWithSimpleColumns()
+    {
+        int numRows = 10;
+        long leastExpectedTimestamp = Timestamp.from(Instant.now()).getTime();
+        Set<Long> observedLMT = new HashSet<>();
+        Tester.builder(TestSchema.builder()
+                                 .withPartitionKey("pk", bridge.aInt())
+                                 .withColumn("a", bridge.text())
+                                 .withColumn("b", bridge.aDouble())
+                                 .withColumn("c", bridge.uuid()))
+              .withLastModifiedTimestampColumn()
+              .dontWriteRandomData()
+              .withDelayBetweenSSTablesInSecs(10)
+              .withSSTableWriter(writer -> {
+                  for (int row = 0; row < numRows; row++)
+                  {
+                      writer.write(row, "text" + row, Math.random(), UUID.randomUUID());
+                  }
+              })
+              .withSSTableWriter(writer -> {
+                  // The second write overrides the first one above
+                  for (int row = 0; row < numRows; row++)
+                  {
+                      // Makes sure the insertion time of each row is unique
+                      Uninterruptibles.sleepUninterruptibly(1, TimeUnit.MILLISECONDS);
+                      writer.write(row, "text" + row, Math.random(), UUID.randomUUID());
+                  }
+              })
+              .withCheck(dataset -> {
+                  for (Row row : dataset.collectAsList())
+                  {
+                      assertEquals(5, row.length());
+                      long lmt = row.getTimestamp(4).getTime();
+                      assertTrue(lmt > leastExpectedTimestamp + 10);
+                      assertTrue("Observed a duplicated LMT", observedLMT.add(lmt));
+                  }
+              })
+              .run();
+    }
+
+    @Test
+    public void testLastModifiedTimestampAddedWithComplexColumns()
+    {
+        long leastExpectedTimestamp = Timestamp.from(Instant.now()).getTime();
+        Set<Long> observedLMT = new HashSet<>();
+        Tester.builder(TestSchema.builder()
+                                 .withPartitionKey("pk", bridge.timeuuid())
+                                 .withClusteringKey("ck", bridge.aInt())
+                                 .withColumn("a", bridge.map(bridge.text(),
+                                                             bridge.set(bridge.text()).frozen()))
+                                 .withColumn("b", bridge.set(bridge.text()))
+                                 .withColumn("c", bridge.tuple(bridge.aInt(),
+                                                               bridge.tuple(bridge.bigint(),
+                                                                            bridge.timeuuid())))
+                                 .withColumn("d", bridge.frozen(bridge.list(bridge.aFloat())))
+                                 .withColumn("e", bridge.udt("keyspace", "udt")
+                                                        .withField("field1", bridge.varchar())
+                                                        .withField("field2", bridge.frozen(bridge.set(bridge.text())))
+                                                        .build()))
+              .withLastModifiedTimestampColumn()
+              .withNumRandomRows(10)
+              .withNumRandomSSTables(2)
+              // Makes sure the insertion time of each row is unique
+              .withWriteListener(row -> Uninterruptibles.sleepUninterruptibly(1, TimeUnit.MILLISECONDS))
+              .withCheck(dataset -> {
+                  for (Row row : dataset.collectAsList())
+                  {
+                      assertEquals(8, row.length());
+                      long lmt = row.getTimestamp(7).getTime();
+                      assertTrue(lmt > leastExpectedTimestamp);
+                      assertTrue("Observed a duplicated LMT", observedLMT.add(lmt));
+                  }
+              })
+              .run();
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/KryoSerializationTests.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/KryoSerializationTests.java
new file mode 100644
index 0000000..9cb6e44
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/KryoSerializationTests.java
@@ -0,0 +1,359 @@
+/*
+ * 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.cassandra.spark;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.UUID;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.commons.lang3.RandomStringUtils;
+import org.junit.Test;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+import org.apache.cassandra.bridge.CassandraVersion;
+import org.apache.cassandra.clients.SslConfig;
+import org.apache.cassandra.spark.data.CqlField;
+import org.apache.cassandra.spark.data.CqlTable;
+import org.apache.cassandra.spark.data.LocalDataLayer;
+import org.apache.cassandra.spark.data.ReplicationFactor;
+import org.apache.cassandra.spark.data.VersionRunner;
+import org.apache.cassandra.spark.data.partitioner.CassandraInstance;
+import org.apache.cassandra.spark.data.partitioner.CassandraRing;
+import org.apache.cassandra.spark.data.partitioner.TokenPartitioner;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.quicktheories.QuickTheory.qt;
+import static org.quicktheories.generators.SourceDSL.arbitrary;
+import static org.quicktheories.generators.SourceDSL.booleans;
+import static org.quicktheories.generators.SourceDSL.integers;
+
+public class KryoSerializationTests extends VersionRunner
+{
+    private static final Kryo KRYO = new Kryo();
+
+    static
+    {
+        new KryoRegister().registerClasses(KRYO);
+    }
+
+    public KryoSerializationTests(CassandraVersion version)
+    {
+        super(version);
+    }
+
+    private static Output serialize(Object object)
+    {
+        try (Output out = new Output(1024, -1))
+        {
+            KRYO.writeObject(out, object);
+            return out;
+        }
+    }
+
+    private static <T> T deserialize(Output output, Class<T> type)
+    {
+        try (Input in = new Input(output.getBuffer(), 0, output.position()))
+        {
+            return KRYO.readObject(in, type);
+        }
+    }
+
+    @Test
+    public void testCqlField()
+    {
+        qt().withExamples(25)
+            .forAll(booleans().all(), booleans().all(), TestUtils.cql3Type(bridge), integers().all())
+            .checkAssert((isPartitionKey, isClusteringKey, cqlType, position) -> {
+                CqlField field = new CqlField(isPartitionKey,
+                                              isClusteringKey && !isPartitionKey,
+                                              false,
+                                              RandomStringUtils.randomAlphanumeric(5, 20),
+                                              cqlType,
+                                              position);
+                Output out = serialize(field);
+                CqlField deserialized = deserialize(out, CqlField.class);
+                assertEquals(field, deserialized);
+                assertEquals(field.name(), deserialized.name());
+                assertEquals(field.type(), deserialized.type());
+                assertEquals(field.position(), deserialized.position());
+                assertEquals(field.isPartitionKey(), deserialized.isPartitionKey());
+                assertEquals(field.isClusteringColumn(), deserialized.isClusteringColumn());
+            });
+    }
+
+    @Test
+    public void testCqlFieldSet()
+    {
+        qt().withExamples(25)
+            .forAll(booleans().all(), booleans().all(), TestUtils.cql3Type(bridge), integers().all())
+            .checkAssert((isPartitionKey, isClusteringKey, cqlType, position) -> {
+                CqlField.CqlSet setType = bridge.set(cqlType);
+                CqlField field = new CqlField(isPartitionKey,
+                                              isClusteringKey && !isPartitionKey,
+                                              false,
+                                              RandomStringUtils.randomAlphanumeric(5, 20),
+                                              setType,
+                                              position);
+                Output out = serialize(field);
+                CqlField deserialized = deserialize(out, CqlField.class);
+                assertEquals(field, deserialized);
+                assertEquals(field.name(), deserialized.name());
+                assertEquals(field.type(), deserialized.type());
+                assertEquals(field.position(), deserialized.position());
+                assertEquals(field.isPartitionKey(), deserialized.isPartitionKey());
+                assertEquals(field.isClusteringColumn(), deserialized.isClusteringColumn());
+            });
+    }
+
+    @Test
+    public void testCqlFieldList()
+    {
+        qt().withExamples(25)
+            .forAll(booleans().all(), booleans().all(), TestUtils.cql3Type(bridge), integers().all())
+            .checkAssert((isPartitionKey, isClusteringKey, cqlType, position) -> {
+                CqlField.CqlList listType = bridge.list(cqlType);
+                CqlField field = new CqlField(isPartitionKey,
+                                              isClusteringKey && !isPartitionKey,
+                                              false,
+                                              RandomStringUtils.randomAlphanumeric(5, 20),
+                                              listType,
+                                              position);
+                Output out = serialize(field);
+                CqlField deserialized = deserialize(out, CqlField.class);
+                assertEquals(field, deserialized);
+                assertEquals(field.name(), deserialized.name());
+                assertEquals(field.type(), deserialized.type());
+                assertEquals(field.position(), deserialized.position());
+                assertEquals(field.isPartitionKey(), deserialized.isPartitionKey());
+                assertEquals(field.isClusteringColumn(), deserialized.isClusteringColumn());
+            });
+    }
+
+    @Test
+    public void testCqlFieldMap()
+    {
+        qt().withExamples(25)
+            .forAll(booleans().all(), booleans().all(), TestUtils.cql3Type(bridge), TestUtils.cql3Type(bridge))
+            .checkAssert((isPartitionKey, isClusteringKey, cqlType1, cqlType2) -> {
+                CqlField.CqlMap mapType = bridge.map(cqlType1, cqlType2);
+                CqlField field = new CqlField(isPartitionKey,
+                                              isClusteringKey && !isPartitionKey,
+                                              false,
+                                              RandomStringUtils.randomAlphanumeric(5, 20),
+                                              mapType,
+                                              2);
+                Output out = serialize(field);
+                CqlField deserialized = deserialize(out, CqlField.class);
+                assertEquals(field, deserialized);
+                assertEquals(field.name(), deserialized.name());
+                assertEquals(field.type(), deserialized.type());
+                assertEquals(field.position(), deserialized.position());
+                assertEquals(field.isPartitionKey(), deserialized.isPartitionKey());
+                assertEquals(field.isClusteringColumn(), deserialized.isClusteringColumn());
+            });
+    }
+
+    @Test
+    public void testCqlUdt()
+    {
+        qt().withExamples(25)
+            .forAll(TestUtils.cql3Type(bridge), TestUtils.cql3Type(bridge))
+            .checkAssert((type1, type2) -> {
+                CqlField.CqlUdt udt = bridge.udt("keyspace", "testudt")
+                                            .withField("a", type1)
+                                            .withField("b", type2)
+                                            .build();
+                CqlField field = new CqlField(false, false, false, RandomStringUtils.randomAlphanumeric(5, 20), udt, 2);
+                Output out = serialize(field);
+                CqlField deserialized = deserialize(out, CqlField.class);
+                assertEquals(field, deserialized);
+                assertEquals(field.name(), deserialized.name());
+                assertEquals(udt, deserialized.type());
+                assertEquals(field.position(), deserialized.position());
+                assertEquals(field.isPartitionKey(), deserialized.isPartitionKey());
+                assertEquals(field.isClusteringColumn(), deserialized.isClusteringColumn());
+            });
+    }
+
+    @Test
+    public void testCqlTuple()
+    {
+        qt().withExamples(25)
+            .forAll(TestUtils.cql3Type(bridge), TestUtils.cql3Type(bridge))
+            .checkAssert((type1, type2) -> {
+                CqlField.CqlTuple tuple = bridge.tuple(type1,
+                                                       bridge.blob(),
+                                                       type2,
+                                                       bridge.set(bridge.text()),
+                                                       bridge.bigint(),
+                                                       bridge.map(type2, bridge.timeuuid()));
+                CqlField field = new CqlField(false, false, false, RandomStringUtils.randomAlphanumeric(5, 20), tuple, 2);
+                Output out = serialize(field);
+                CqlField deserialized = deserialize(out, CqlField.class);
+                assertEquals(field, deserialized);
+                assertEquals(field.name(), deserialized.name());
+                assertEquals(tuple, deserialized.type());
+                assertEquals(field.position(), deserialized.position());
+                assertEquals(field.isPartitionKey(), deserialized.isPartitionKey());
+                assertEquals(field.isClusteringColumn(), deserialized.isClusteringColumn());
+            });
+    }
+
+    @Test
+    public void testCqlTable()
+    {
+        List<CqlField> fields = ImmutableList.of(new CqlField(true, false, false, "a", bridge.bigint(), 0),
+                                                 new CqlField(true, false, false, "b", bridge.bigint(), 1),
+                                                 new CqlField(false, true, false, "c", bridge.bigint(), 2),
+                                                 new CqlField(false, false, false, "d", bridge.timestamp(), 3),
+                                                 new CqlField(false, false, false, "e", bridge.text(), 4));
+        ReplicationFactor replicationFactor = new ReplicationFactor(ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy,
+                                                                    ImmutableMap.of("DC1", 3, "DC2", 3));
+        CqlTable table = new CqlTable("test_keyspace",
+                                      "test_table",
+                                      "create table test_keyspace.test_table"
+                                    + " (a bigint, b bigint, c bigint, d bigint, e bigint, primary key((a, b), c));",
+                                      replicationFactor,
+                                      fields);
+
+        Output out = serialize(table);
+        CqlTable deserialized = deserialize(out, CqlTable.class);
+        assertNotNull(deserialized);
+        assertEquals(table, deserialized);
+    }
+
+    @Test
+    public void testCassandraInstance()
+    {
+        CassandraInstance instance = new CassandraInstance("-9223372036854775807", "local1-i1", "DC1");
+        Output out = serialize(instance);
+        CassandraInstance deserialized = deserialize(out, CassandraInstance.class);
+        assertNotNull(deserialized);
+        assertEquals(instance, deserialized);
+    }
+
+    @Test
+    public void testCassandraRing()
+    {
+        qt().forAll(TestUtils.partitioners())
+            .checkAssert(partitioner -> {
+                CassandraRing ring = TestUtils.createRing(partitioner, ImmutableMap.of("DC1", 3, "DC2", 3));
+                Output out = serialize(ring);
+                CassandraRing deserialized = deserialize(out, CassandraRing.class);
+                assertNotNull(deserialized);
+                assertEquals(ring, deserialized);
+                assertEquals(partitioner, deserialized.partitioner());
+            });
+    }
+
+    @Test
+    public void testLocalDataLayer()
+    {
+        String path1 = UUID.randomUUID().toString();
+        String path2 = UUID.randomUUID().toString();
+        String path3 = UUID.randomUUID().toString();
+        LocalDataLayer localDataLayer = new LocalDataLayer(bridge.getVersion(),
+                                                           "test_keyspace",
+                                                           "create table test_keyspace.test_table"
+                                                         + " (a int, b int, c int, primary key(a, b));",
+                                                           path1,
+                                                           path2,
+                                                           path3);
+        Output out = serialize(localDataLayer);
+        LocalDataLayer deserialized = deserialize(out, LocalDataLayer.class);
+        assertNotNull(deserialized);
+        assertEquals(localDataLayer.version(), deserialized.version());
+        assertEquals(localDataLayer, deserialized);
+    }
+
+    @Test
+    public void testTokenPartitioner()
+    {
+        qt().forAll(TestUtils.partitioners(),
+                    arbitrary().pick(Arrays.asList(3, 16, 128)),
+                    arbitrary().pick(Arrays.asList(1, 4, 16)),
+                    arbitrary().pick(Arrays.asList(4, 16, 64)))
+            .checkAssert((partitioner, numInstances, defaultParallelism, numCores) -> {
+                CassandraRing ring = TestUtils.createRing(partitioner, numInstances);
+                TokenPartitioner tokenPartitioner = new TokenPartitioner(ring, defaultParallelism, numCores);
+                Output out = serialize(tokenPartitioner);
+                TokenPartitioner deserialized = deserialize(out, TokenPartitioner.class);
+                assertNotNull(deserialized);
+                assertEquals(tokenPartitioner.numPartitions(), deserialized.numPartitions());
+                assertEquals(tokenPartitioner.subRanges().size(), deserialized.subRanges().size());
+                for (int index = 0; index < tokenPartitioner.subRanges().size(); index++)
+                {
+                    assertEquals(tokenPartitioner.subRanges().get(index), deserialized.subRanges().get(index));
+                }
+                assertEquals(tokenPartitioner.ring(), deserialized.ring());
+            });
+    }
+
+    @Test
+    public void testCqlUdtField()
+    {
+        CqlField.CqlUdt udt = bridge.udt("udt_keyspace", "udt_table")
+                                    .withField("c", bridge.text())
+                                    .withField("b", bridge.timestamp())
+                                    .withField("a", bridge.bigint())
+                                    .build();
+        Output out = new Output(1024, -1);
+        udt.write(out);
+        out.close();
+        Input in = new Input(out.getBuffer(), 0, out.position());
+        CqlField.CqlUdt deserialized = (CqlField.CqlUdt) CqlField.CqlType.read(in, bridge);
+        assertEquals(udt, deserialized);
+        for (int index = 0; index < deserialized.fields().size(); index++)
+        {
+            assertEquals(udt.field(index), deserialized.field(index));
+        }
+    }
+
+    @Test
+    public void testSslConfig()
+    {
+        SslConfig config = new SslConfig.Builder<>()
+                                     .keyStorePath("keyStorePath")
+                                     .base64EncodedKeyStore("encodedKeyStore")
+                                     .keyStorePassword("keyStorePassword")
+                                     .keyStoreType("keyStoreType")
+                                     .trustStorePath("trustStorePath")
+                                     .base64EncodedTrustStore("encodedTrustStore")
+                                     .trustStorePassword("trustStorePassword")
+                                     .trustStoreType("trustStoreType")
+                                     .build();
+        Output out = serialize(config);
+        SslConfig deserialized = deserialize(out, SslConfig.class);
+
+        assertEquals(config.keyStorePath(), deserialized.keyStorePath());
+        assertEquals(config.base64EncodedKeyStore(), deserialized.base64EncodedKeyStore());
+        assertEquals(config.keyStorePassword(), deserialized.keyStorePassword());
+        assertEquals(config.keyStoreType(), deserialized.keyStoreType());
+        assertEquals(config.trustStorePath(), deserialized.trustStorePath());
+        assertEquals(config.base64EncodedTrustStore(), deserialized.base64EncodedTrustStore());
+        assertEquals(config.trustStorePassword(), deserialized.trustStorePassword());
+        assertEquals(config.trustStoreType(), deserialized.trustStoreType());
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/TestDataLayer.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/TestDataLayer.java
new file mode 100644
index 0000000..96267e6
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/TestDataLayer.java
@@ -0,0 +1,159 @@
+/*
+ * 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.cassandra.spark;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.nio.file.Path;
+import java.util.Collection;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.stream.Collectors;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+import org.apache.cassandra.bridge.CassandraBridge;
+import org.apache.cassandra.bridge.CassandraBridgeFactory;
+import org.apache.cassandra.bridge.CassandraVersion;
+import org.apache.cassandra.spark.cdc.CommitLogProvider;
+import org.apache.cassandra.spark.cdc.TableIdLookup;
+import org.apache.cassandra.spark.data.BasicSupplier;
+import org.apache.cassandra.spark.data.CqlTable;
+import org.apache.cassandra.spark.data.DataLayer;
+import org.apache.cassandra.spark.data.SSTablesSupplier;
+import org.apache.cassandra.spark.data.partitioner.Partitioner;
+import org.apache.cassandra.spark.sparksql.filters.PartitionKeyFilter;
+import org.apache.cassandra.spark.sparksql.filters.SparkRangeFilter;
+import org.apache.cassandra.spark.utils.test.TestSSTable;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+public class TestDataLayer extends DataLayer
+{
+    public static final ExecutorService FILE_IO_EXECUTOR =
+            Executors.newFixedThreadPool(1, new ThreadFactoryBuilder().setNameFormat("test-file-io-%d")
+                                                                      .setDaemon(true)
+                                                                      .build());
+
+    @NotNull
+    CassandraBridge bridge;
+    @NotNull
+    Collection<Path> dataDbFiles;
+    @Nullable
+    CqlTable table;
+    final String jobId;
+
+    public TestDataLayer(@NotNull CassandraBridge bridge,
+                         @NotNull Collection<Path> dataDbFiles,
+                         @Nullable CqlTable table)
+    {
+        this.bridge = bridge;
+        this.dataDbFiles = dataDbFiles;
+        this.table = table;
+        this.jobId = UUID.randomUUID().toString();
+    }
+
+    @Override
+    public CassandraBridge bridge()
+    {
+        return bridge;
+    }
+
+    @Override
+    public int partitionCount()
+    {
+        return 0;
+    }
+
+    @Override
+    public CqlTable cqlTable()
+    {
+        return table;
+    }
+
+    @Override
+    public boolean isInPartition(int partitionId, BigInteger token, ByteBuffer key)
+    {
+        return true;
+    }
+
+    @Override
+    public CommitLogProvider commitLogs(int partitionId)
+    {
+        throw new UnsupportedOperationException("Test CommitLogProvider not implemented yet");
+    }
+
+    @Override
+    public TableIdLookup tableIdLookup()
+    {
+        throw new UnsupportedOperationException("Test TableIdLookup not implemented yet");
+    }
+
+    @Override
+    protected ExecutorService executorService()
+    {
+        return FILE_IO_EXECUTOR;
+    }
+
+    @Override
+    @NotNull
+    public SSTablesSupplier sstables(int partitionId,
+                                     @Nullable SparkRangeFilter sparkRangeFilter,
+                                     @NotNull List<PartitionKeyFilter> partitionKeyFilters)
+    {
+        return new BasicSupplier(dataDbFiles.stream()
+                                            .map(TestSSTable::at)
+                                            .collect(Collectors.toSet()));
+    }
+
+    @Override
+    public Partitioner partitioner()
+    {
+        return Partitioner.Murmur3Partitioner;
+    }
+
+    @Override
+    public String jobId()
+    {
+        return jobId;
+    }
+
+    private void writeObject(ObjectOutputStream out) throws IOException
+    {
+        // Falling back to JDK serialization
+        out.writeObject(version());
+        out.writeObject(dataDbFiles);
+        bridge.javaSerialize(out, table);  // Delegate (de-)serialization of version-specific objects to the Cassandra Bridge
+    }
+
+    @SuppressWarnings("unchecked")
+    private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException
+    {
+        // Falling back to JDK deserialization
+        bridge = CassandraBridgeFactory.get((CassandraVersion) in.readObject());
+        dataDbFiles = (Collection<Path>) in.readObject();
+        table = bridge.javaDeserialize(in, CqlTable.class);  // Delegate (de-)serialization of version-specific objects to the Cassandra Bridge
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/TestRunnable.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/TestRunnable.java
new file mode 100644
index 0000000..6dbbc37
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/TestRunnable.java
@@ -0,0 +1,31 @@
+/*
+ * 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.cassandra.spark;
+
+import java.io.IOException;
+import java.nio.file.Path;
+
+import org.apache.cassandra.bridge.CassandraBridge;
+import org.apache.cassandra.spark.data.partitioner.Partitioner;
+
+public interface TestRunnable
+{
+    void run(Partitioner partitioner, Path directory, CassandraBridge bridge) throws IOException;
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/TestUtils.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/TestUtils.java
new file mode 100644
index 0000000..85b14f3
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/TestUtils.java
@@ -0,0 +1,355 @@
+/*
+ * 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.cassandra.spark;
+
+import java.io.IOException;
+import java.math.BigInteger;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.commons.io.FileUtils;
+
+import org.apache.cassandra.bridge.CassandraBridge;
+import org.apache.cassandra.bridge.CassandraBridgeFactory;
+import org.apache.cassandra.bridge.CassandraVersion;
+import org.apache.cassandra.spark.config.SchemaFeatureSet;
+import org.apache.cassandra.spark.data.CqlField;
+import org.apache.cassandra.spark.data.FileType;
+import org.apache.cassandra.spark.data.ReplicationFactor;
+import org.apache.cassandra.spark.data.partitioner.CassandraInstance;
+import org.apache.cassandra.spark.data.partitioner.CassandraRing;
+import org.apache.cassandra.spark.data.partitioner.Partitioner;
+import org.apache.cassandra.spark.utils.FilterUtils;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.streaming.OutputMode;
+import org.apache.spark.sql.streaming.StreamingQuery;
+import org.apache.spark.sql.types.StructType;
+import org.jetbrains.annotations.Nullable;
+import org.quicktheories.core.Gen;
+
+import static org.junit.Assert.assertTrue;
+import static org.quicktheories.QuickTheory.qt;
+import static org.quicktheories.generators.SourceDSL.arbitrary;
+
+public final class TestUtils
+{
+    private static final SparkSession SPARK = SparkSession.builder()
+                                                          .appName("Java Test")
+                                                          .config("spark.master", "local")
+                                                          .getOrCreate();
+
+    private TestUtils()
+    {
+        throw new IllegalStateException(getClass() + " is static utility class and shall not be instantiated");
+    }
+
+    public static long countSSTables(Path directory) throws IOException
+    {
+        return getFileType(directory, FileType.DATA).count();
+    }
+
+    public static Path getFirstFileType(Path directory, FileType fileType) throws IOException
+    {
+        return getFileType(directory, fileType).findFirst().orElseThrow(() ->
+                new IllegalStateException(String.format("Could not find %s file", fileType.getFileSuffix())));
+    }
+
+    public static Stream<Path> getFileType(Path directory, FileType fileType) throws IOException
+    {
+        return Files.list(directory)
+                    .filter(path -> path.getFileName().toString().endsWith("-" + fileType.getFileSuffix()));
+    }
+
+    /**
+     * Run test for all supported Cassandra versions
+     *
+     * @param test unit test
+     */
+    public static void runTest(TestRunnable test)
+    {
+        qt().forAll(TestUtils.partitioners(), TestUtils.bridges())
+            .checkAssert((partitioner, bridge) -> TestUtils.runTest(partitioner, bridge, test));
+    }
+
+    public static void runTest(CassandraVersion version, TestRunnable test)
+    {
+        qt().forAll(TestUtils.partitioners())
+            .checkAssert(partitioner -> TestUtils.runTest(partitioner, version, test));
+    }
+
+    public static void runTest(Partitioner partitioner, CassandraVersion version, TestRunnable test)
+    {
+        runTest(partitioner, CassandraBridgeFactory.get(version), test);
+    }
+
+    /**
+     * Create tmp directory and clean up after test
+     *
+     * @param bridge cassandra bridge
+     * @param test   unit test
+     */
+    public static void runTest(Partitioner partitioner, CassandraBridge bridge, TestRunnable test)
+    {
+        Path directory = null;
+        try
+        {
+            directory = Files.createTempDirectory(UUID.randomUUID().toString());
+            test.run(partitioner, directory, bridge);
+        }
+        catch (IOException exception)
+        {
+            throw new RuntimeException(exception);
+        }
+        finally
+        {
+            if (directory != null)
+            {
+                try
+                {
+                    FileUtils.deleteDirectory(directory.toFile());
+                }
+                catch (IOException ignore)
+                {
+                }
+            }
+        }
+    }
+
+    // CHECKSTYLE IGNORE: Method with many parameters
+    public static StreamingQuery openStreaming(String keyspace,
+                                               String createStmt,
+                                               CassandraVersion version,
+                                               Partitioner partitioner,
+                                               Path dir,
+                                               Path outputDir,
+                                               Path checkpointDir,
+                                               String dataSourceFQCN,
+                                               boolean addLastModificationTime)
+    {
+        Dataset<Row> rows = SPARK.readStream()
+                                 .format(dataSourceFQCN)
+                                 .option("keyspace", keyspace)
+                                 .option("createStmt", createStmt)
+                                 .option("dirs", dir.toAbsolutePath().toString())
+                                 .option("version", version.toString())
+                                 .option("useSSTableInputStream", true)  // Use in the test system to test the SSTableInputStream
+                                 .option("partitioner", partitioner.name())
+                                 .option(SchemaFeatureSet.LAST_MODIFIED_TIMESTAMP.optionName(), addLastModificationTime)
+                                 .option(SchemaFeatureSet.UPDATED_FIELDS_INDICATOR.optionName(), true)   // Always add the indicator column for CDC
+                                 .option(SchemaFeatureSet.UPDATE_FLAG.optionName(), true)                // Always add the update flag for CDC
+                                 .option(SchemaFeatureSet.CELL_DELETION_IN_COMPLEX.optionName(),  true)  // Support tombstones in complex for CDC
+                                 .option(SchemaFeatureSet.RANGE_DELETION.optionName(), true)             // Support range tombstones for CDC
+                                 .option("udts", "")
+                                 .load();
+        try
+        {
+            return rows.writeStream()
+                       .format("parquet")
+                       .option("path", outputDir.toString())
+                       .option("checkpointLocation", checkpointDir.toString())
+                       .outputMode(OutputMode.Append())
+                       .start();
+        }
+        catch (Exception exception)
+        {
+            // In Spark3 start() can throw a TimeoutException
+            throw new RuntimeException(exception);
+        }
+    }
+
+    public static Dataset<Row> read(Path path, StructType schema)
+    {
+        return SPARK.read()
+                    .format("parquet")
+                    .option("path", path.toString())
+                    .schema(schema)
+                    .load();
+    }
+
+    // CHECKSTYLE IGNORE: Method with many parameters
+    public static Dataset<Row> openLocalDataset(Partitioner partitioner,
+                                                Path directory,
+                                                String keyspace,
+                                                String createStatement,
+                                                CassandraVersion version,
+                                                Set<CqlField.CqlUdt> udts,
+                                                boolean addLastModifiedTimestampColumn,
+                                                @Nullable String statsClass,
+                                                @Nullable String filterExpression,
+                                                @Nullable String... columns)
+    {
+        DataFrameReader frameReader = SPARK.read().format("org.apache.cassandra.spark.sparksql.LocalDataSource")
+                .option("keyspace", keyspace)
+                .option("createStmt", createStatement)
+                .option("dirs", directory.toAbsolutePath().toString())
+                .option("version", version.toString())
+                .option("useSSTableInputStream", true)  // Use in the test system to test the SSTableInputStream
+                .option("partitioner", partitioner.name())
+                .option(SchemaFeatureSet.LAST_MODIFIED_TIMESTAMP.optionName(), addLastModifiedTimestampColumn)
+                .option("udts", udts.stream()
+                                    .map(udt -> udt.createStatement(keyspace))
+                                    .collect(Collectors.joining("\n")));
+        if (statsClass != null)
+        {
+            frameReader = frameReader.option("statsClass", statsClass);
+        }
+        Dataset<Row> dataset = frameReader.load();
+        if (filterExpression != null)
+        {
+            // Attach partition filter criteria
+            dataset = dataset.filter(filterExpression);
+        }
+        if (columns != null && columns.length > 0)
+        {
+            // Attach column select criteria
+            if (columns.length == 1)
+            {
+                dataset = dataset.select(columns[0]);
+            }
+            else
+            {
+                dataset = dataset.select(columns[0], Arrays.copyOfRange(columns, 1, columns.length));
+            }
+        }
+        return dataset;
+    }
+
+    public static ReplicationFactor simpleStrategy()
+    {
+        return new ReplicationFactor(ReplicationFactor.ReplicationStrategy.SimpleStrategy, ImmutableMap.of("DC1", 3));
+    }
+
+    public static ReplicationFactor networkTopologyStrategy()
+    {
+        return networkTopologyStrategy(ImmutableMap.of("DC1", 3));
+    }
+
+    public static ReplicationFactor networkTopologyStrategy(Map<String, Integer> options)
+    {
+        return new ReplicationFactor(ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy, options);
+    }
+
+    /* Quick Theories Helpers */
+
+    public static Gen<CassandraVersion> versions()
+    {
+        return arbitrary().pick(CassandraVersion.implementedVersions());
+    }
+
+    public static Gen<CassandraBridge> bridges()
+    {
+        return arbitrary().pick(testableVersions().stream()
+                                                  .map(CassandraBridgeFactory::get)
+                                                  .collect(Collectors.toList()));
+    }
+
+    public static List<CassandraVersion> testableVersions()
+    {
+        return ImmutableList.copyOf(CassandraVersion.implementedVersions());
+    }
+
+    public static Gen<CqlField.NativeType> cql3Type(CassandraBridge bridge)
+    {
+        return arbitrary().pick(bridge.supportedTypes());
+    }
+
+    public static Gen<CqlField.SortOrder> sortOrder()
+    {
+        return arbitrary().enumValues(CqlField.SortOrder.class);
+    }
+
+    public static Gen<CassandraVersion> tombstoneVersions()
+    {
+        return arbitrary().pick(tombstoneTestableVersions());
+    }
+
+    public static List<CassandraVersion> tombstoneTestableVersions()
+    {
+        // Tombstone SSTable writing and SSTable-to-JSON conversion are not implemented for Cassandra version 3.0
+        return ImmutableList.of(CassandraVersion.FOURZERO);
+    }
+
+    public static Gen<Partitioner> partitioners()
+    {
+        return arbitrary().enumValues(Partitioner.class);
+    }
+
+    public static CassandraRing createRing(Partitioner partitioner, int numInstances)
+    {
+        return createRing(partitioner, ImmutableMap.of("DC1", numInstances));
+    }
+
+    public static CassandraRing createRing(Partitioner partitioner, Map<String, Integer> numInstances)
+    {
+        Collection<CassandraInstance> instances = numInstances.entrySet().stream()
+                .map(dataCenter -> TestUtils.createInstances(partitioner, dataCenter.getValue(), dataCenter.getKey()))
+                .flatMap(Collection::stream)
+                .collect(Collectors.toList());
+        Map<String, Integer> dataCenters = numInstances.entrySet().stream()
+                .collect(Collectors.toMap(Map.Entry::getKey, dataCenter -> Math.min(dataCenter.getValue(), 3)));
+        return new CassandraRing(partitioner, "test", new ReplicationFactor(
+                ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy, dataCenters), instances);
+    }
+
+    public static Collection<CassandraInstance> createInstances(Partitioner partitioner,
+                                                                int numInstances,
+                                                                String dataCenter)
+    {
+        Preconditions.checkArgument(numInstances > 0, "NumInstances must be greater than zero");
+        BigInteger split = partitioner.maxToken()
+                                      .subtract(partitioner.minToken())
+                                      .divide(BigInteger.valueOf(numInstances));
+        Collection<CassandraInstance> instances = new ArrayList<>(numInstances);
+        BigInteger token = partitioner.minToken();
+        for (int instance = 0; instance < numInstances; instance++)
+        {
+            instances.add(new CassandraInstance(token.toString(), "local-i" + instance, dataCenter));
+            token = token.add(split);
+            assertTrue(token.compareTo(partitioner.maxToken()) <= 0);
+        }
+        return instances;
+    }
+
+    public static Set<String> getKeys(List<List<String>> values)
+    {
+        Set<String> filterKeys = new HashSet<>();
+        FilterUtils.cartesianProduct(values).forEach(keys -> {
+            String compositeKey = String.join(":", keys);
+            filterKeys.add(compositeKey);
+        });
+        return filterKeys;
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/Tester.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/Tester.java
new file mode 100644
index 0000000..a7daaab
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/Tester.java
@@ -0,0 +1,547 @@
+/*
+ * 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.cassandra.spark;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.commons.lang3.mutable.MutableLong;
+
+import org.apache.cassandra.bridge.CassandraBridge;
+import org.apache.cassandra.bridge.CassandraVersion;
+import org.apache.cassandra.spark.utils.test.TestSchema;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+import org.quicktheories.core.Gen;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.quicktheories.QuickTheory.qt;
+import static org.quicktheories.generators.SourceDSL.arbitrary;
+
+public final class Tester
+{
+    static final int DEFAULT_NUM_ROWS = 200;
+
+    @NotNull
+    private final List<CassandraVersion> versions;
+    @Nullable
+    private final TestSchema.Builder schemaBuilder;
+    @Nullable
+    private final Function<String, TestSchema.Builder> schemaBuilderFunc;
+    private final int numRandomRows;
+    private final int expectedRowCount;
+    @NotNull
+    private final List<Consumer<TestSchema.TestRow>> writeListeners;
+    @NotNull
+    private final List<Consumer<TestSchema.TestRow>> readListeners;
+    @NotNull
+    private final List<Writer> writers;
+    @NotNull
+    private final List<Consumer<Dataset<Row>>> checks;
+    @NotNull
+    private final List<Integer> numSSTables;
+    @Nullable
+    private final Runnable reset;
+    @Nullable
+    private final String filterExpression;
+    @Nullable
+    private final String[] columns;
+    @NotNull
+    private final Set<String> sumFields;
+    private final boolean shouldCheckNumSSTables;
+    private final boolean addLastModifiedTimestamp;
+    private final int delayBetweenSSTablesInSecs;
+    private final String statsClass;
+    private final boolean upsert;
+
+    // CHECKSTYLE IGNORE: Constructor with many parameters
+    private Tester(@NotNull List<CassandraVersion> versions,
+                   @Nullable TestSchema.Builder schemaBuilder,
+                   @Nullable Function<String, TestSchema.Builder> schemaBuilderFunc,
+                   @NotNull List<Integer> numSSTables,
+                   @NotNull List<Consumer<TestSchema.TestRow>> writeListeners,
+                   @NotNull List<Consumer<TestSchema.TestRow>> readListeners,
+                   @NotNull List<Writer> writers,
+                   @NotNull List<Consumer<Dataset<Row>>> checks,
+                   @NotNull Set<String> sumFields,
+                   @Nullable Runnable reset,
+                   @Nullable String filterExpression,
+                   int numRandomRows, int expectedRowCount,
+                   boolean shouldCheckNumSSTables,
+                   @Nullable String[] columns,
+                   boolean addLastModifiedTimestamp,
+                   int delayBetweenSSTablesInSecs,
+                   @Nullable String statsClass,
+                   boolean upsert)
+    {
+        this.versions = versions;
+        this.schemaBuilder = schemaBuilder;
+        this.schemaBuilderFunc = schemaBuilderFunc;
+        this.numSSTables = numSSTables;
+        this.writeListeners = writeListeners;
+        this.readListeners = readListeners;
+        this.writers = writers;
+        this.checks = checks;
+        this.sumFields = sumFields;
+        this.reset = reset;
+        this.filterExpression = filterExpression;
+        this.numRandomRows = numRandomRows;
+        this.expectedRowCount = expectedRowCount;
+        this.shouldCheckNumSSTables = shouldCheckNumSSTables;
+        this.columns = columns;
+        this.addLastModifiedTimestamp = addLastModifiedTimestamp;
+        this.delayBetweenSSTablesInSecs = delayBetweenSSTablesInSecs;
+        this.statsClass = statsClass;
+        this.upsert = upsert;
+    }
+
+    static Builder builder(@NotNull TestSchema.Builder schemaBuilder)
+    {
+        return new Builder(schemaBuilder);
+    }
+
+    static Builder builder(@NotNull Function<String, TestSchema.Builder> schemaBuilderFunc)
+    {
+        return new Builder(schemaBuilderFunc);
+    }
+
+    static class Writer
+    {
+        final Consumer<CassandraBridge.Writer> consumer;
+        final boolean isTombstoneWriter;
+
+        Writer(Consumer<CassandraBridge.Writer> consumer)
+        {
+            this(consumer, false);
+        }
+
+        Writer(Consumer<CassandraBridge.Writer> consumer, boolean isTombstoneWriter)
+        {
+            this.consumer = consumer;
+            this.isTombstoneWriter = isTombstoneWriter;
+        }
+    }
+
+    public static final class Builder
+    {
+        // TODO: Make use of TestUtils.testableVersions() instead
+        @NotNull
+        private List<CassandraVersion> versions = ImmutableList.of(CassandraVersion.FOURZERO);
+        @Nullable
+        private TestSchema.Builder schemaBuilder;
+        @Nullable
+        private Function<String, TestSchema.Builder> schemaBuilderFunc;
+        private int numRandomRows = DEFAULT_NUM_ROWS;
+        private int expectedRowCount = -1;
+        @NotNull
+        private final List<Consumer<TestSchema.TestRow>> writeListeners = new ArrayList<>();
+        @NotNull
+        private final List<Consumer<TestSchema.TestRow>> readListeners = new ArrayList<>();
+        @NotNull
+        private final List<Writer> writers = new ArrayList<>();
+        @NotNull
+        private final List<Consumer<Dataset<Row>>> checks = new ArrayList<>();
+        @Nullable
+        private Runnable reset = null;
+        @NotNull
+        private List<Integer> numSSTables = ImmutableList.of(1, 2, 5);
+        @NotNull
+        private Set<String> sumFields = Collections.emptySet();
+        @Nullable
+        private String filterExpression;
+        @Nullable
+        private String[] columns = null;
+        private boolean shouldCheckNumSSTables = true;
+        private boolean addLastModifiedTimestamp = false;
+        private int delayBetweenSSTablesInSecs = 0;
+        private String statsClass = null;
+        private boolean upsert = false;
+
+        private Builder(@NotNull TestSchema.Builder schemaBuilder)
+        {
+            this.schemaBuilder = schemaBuilder;
+        }
+
+        private Builder(@NotNull Function<String, TestSchema.Builder> schemaBuilderFunc)
+        {
+            this.schemaBuilderFunc = schemaBuilderFunc;
+        }
+
+        // Runs a test for every Cassandra version given
+        Builder withVersions(@NotNull Collection<CassandraVersion> versions)
+        {
+            this.versions = ImmutableList.copyOf(versions);
+            return this;
+        }
+
+        // Runs a test for every number of SSTables given
+        Builder withNumRandomSSTables(Integer... numSSTables)
+        {
+            this.numSSTables = ImmutableList.copyOf(numSSTables);
+            return this;
+        }
+
+        Builder withSumField(String... fields)
+        {
+            sumFields = ImmutableSet.copyOf(fields);
+            return this;
+        }
+
+        Builder withNumRandomRows(int numRow)
+        {
+            numRandomRows = numRow;
+            return this;
+        }
+
+        Builder dontWriteRandomData()
+        {
+            numSSTables = ImmutableList.of(0);
+            numRandomRows = 0;
+            return this;
+        }
+
+        Builder withWriteListener(@Nullable Consumer<TestSchema.TestRow> writeListener)
+        {
+            if (writeListener != null)
+            {
+                writeListeners.add(writeListener);
+            }
+            return this;
+        }
+
+        Builder withReadListener(@Nullable Consumer<TestSchema.TestRow> readListener)
+        {
+            if (readListener != null)
+            {
+                readListeners.add(readListener);
+            }
+            return this;
+        }
+
+        Builder withSSTableWriter(@Nullable Consumer<CassandraBridge.Writer> consumer)
+        {
+            if (consumer != null)
+            {
+                writers.add(new Writer(consumer));
+            }
+            return this;
+        }
+
+        Builder withTombstoneWriter(@Nullable Consumer<CassandraBridge.Writer> consumer)
+        {
+            if (consumer != null)
+            {
+                writers.add(new Writer(consumer, true));
+            }
+            return this;
+        }
+
+        Builder withCheck(@Nullable Consumer<Dataset<Row>> check)
+        {
+            if (check != null)
+            {
+                checks.add(check);
+            }
+            return this;
+        }
+
+        Builder withExpectedRowCountPerSSTable(int expectedRowCount)
+        {
+            this.expectedRowCount = expectedRowCount;
+            return this;
+        }
+
+        Builder withReset(Runnable reset)
+        {
+            this.reset = reset;
+            return this;
+        }
+
+        Builder withFilter(@NotNull String filterExpression)
+        {
+            this.filterExpression = filterExpression;
+            return this;
+        }
+
+        Builder withColumns(@NotNull String... columns)
+        {
+            this.columns = columns;
+            return this;
+        }
+
+        Builder dontCheckNumSSTables()
+        {
+            shouldCheckNumSSTables = false;
+            return this;
+        }
+
+        Builder withLastModifiedTimestampColumn()
+        {
+            addLastModifiedTimestamp = true;
+            return this;
+        }
+
+        Builder withDelayBetweenSSTablesInSecs(int delay)
+        {
+            delayBetweenSSTablesInSecs = delay;
+            return this;
+        }
+
+        Builder withStatsClass(String statsClass)
+        {
+            this.statsClass = statsClass;
+            return this;
+        }
+
+        public Builder withUpsert()
+        {
+            upsert = true;
+            return this;
+        }
+
+        void run()
+        {
+            Preconditions.checkArgument(schemaBuilder != null || schemaBuilderFunc != null);
+            new Tester(versions,
+                       schemaBuilder,
+                       schemaBuilderFunc,
+                       numSSTables,
+                       writeListeners,
+                       readListeners,
+                       writers,
+                       checks,
+                       sumFields,
+                       reset,
+                       filterExpression,
+                       numRandomRows,
+                       expectedRowCount,
+                       shouldCheckNumSSTables,
+                       columns,
+                       addLastModifiedTimestamp,
+                       delayBetweenSSTablesInSecs,
+                       statsClass,
+                       upsert).run();
+        }
+    }
+
+    private void run()
+    {
+        qt().forAll(versions(), numSSTables())
+            .checkAssert(this::run);
+    }
+
+    private Gen<CassandraVersion> versions()
+    {
+        return arbitrary().pick(versions);
+    }
+
+    private Gen<Integer> numSSTables()
+    {
+        return arbitrary().pick(numSSTables);
+    }
+
+    // CHECKSTYLE IGNORE: Long method
+    private void run(CassandraVersion version, int numSSTables)
+    {
+        TestUtils.runTest(version, (partitioner, directory, bridge) -> {
+            String keyspace = "keyspace_" + UUID.randomUUID().toString().replaceAll("-", "");
+            TestSchema schema = schemaBuilder != null ? schemaBuilder.withKeyspace(keyspace).build()
+                                                      : schemaBuilderFunc.apply(keyspace).build();
+            schema.setCassandraVersion(version);
+
+            // Write SSTables with random data
+            Map<String, MutableLong> sum = sumFields.stream()
+                                                    .collect(Collectors.toMap(Function.identity(),
+                                                                              ignore -> new MutableLong()));
+            Map<String, TestSchema.TestRow> rows = new HashMap<>(numRandomRows);
+            IntStream.range(0, numSSTables).forEach(ssTable ->
+                    schema.writeSSTable(directory, bridge, partitioner, upsert, writer ->
+                             IntStream.range(0, numRandomRows).forEach(row -> {
+                                 TestSchema.TestRow testRow;
+                                 do
+                                 {
+                                     testRow = schema.randomRow();
+                                 }
+                                 while (rows.containsKey(testRow.getKey()));  // Don't write duplicate rows
+
+                                 for (Consumer<TestSchema.TestRow> writeListener : writeListeners)
+                                 {
+                                     writeListener.accept(testRow);
+                                 }
+
+                                 for (String sumField : sumFields)
+                                 {
+                                     sum.get(sumField).add((Number) testRow.get(sumField));
+                                 }
+                                 rows.put(testRow.getKey(), testRow);
+
+                                 Object[] values = testRow.allValues();
+                                 if (upsert)
+                                 {
+                                     rotate(values, schema.partitionKeys.size() + schema.clusteringKeys.size());
+                                 }
+                                 writer.write(values);
+                             })));
+            int sstableCount = numSSTables;
+
+            // Write any custom SSTables e.g. overwriting existing data or tombstones
+            for (Writer writer : writers)
+            {
+                if (sstableCount != 0)
+                {
+                    try
+                    {
+                        TimeUnit.SECONDS.sleep(delayBetweenSSTablesInSecs);
+                    }
+                    catch (InterruptedException exception)
+                    {
+                        throw new RuntimeException(exception.getMessage());
+                    }
+                }
+                if (writer.isTombstoneWriter)
+                {
+                    schema.writeTombstoneSSTable(directory, bridge, partitioner, writer.consumer);
+                }
+                else
+                {
+                    schema.writeSSTable(directory, bridge, partitioner, false, writer.consumer);
+                }
+                sstableCount++;
+            }
+
+            if (shouldCheckNumSSTables)
+            {
+                assertEquals("Number of SSTables written does not match expected",
+                             sstableCount, TestUtils.countSSTables(directory));
+            }
+
+            Dataset<Row> dataset = TestUtils.openLocalDataset(partitioner,
+                                                              directory,
+                                                              schema.keyspace,
+                                                              schema.createStatement,
+                                                              version,
+                                                              schema.udts,
+                                                              addLastModifiedTimestamp,
+                                                              statsClass,
+                                                              filterExpression,
+                                                              columns);
+            int rowCount = 0;
+            Set<String> requiredColumns = columns != null ? new HashSet<>(Arrays.asList(columns)) : null;
+            for (Row row : dataset.collectAsList())
+            {
+                if (requiredColumns != null)
+                {
+                    Set<String> actualColumns = new HashSet<>(Arrays.asList(row.schema().fieldNames()));
+                    assertEquals("Actual Columns and Required Columns should be the same",
+                                 actualColumns, requiredColumns);
+                }
+
+                TestSchema.TestRow actualRow = schema.toTestRow(row, requiredColumns);
+                if (numRandomRows > 0)
+                {
+                    // If we wrote random data, verify values exist
+                    String key = actualRow.getKey();
+                    assertTrue("Unexpected row read in Spark", rows.containsKey(key));
+                    assertEquals("Row read in Spark does not match expected",
+                                 rows.get(key).withColumns(requiredColumns), actualRow);
+                }
+
+                for (Consumer<TestSchema.TestRow> readListener : readListeners)
+                {
+                    readListener.accept(actualRow);
+                }
+                rowCount++;
+            }
+            if (expectedRowCount >= 0)
+            {
+                assertEquals("Number of rows read does not match expected", expectedRowCount * sstableCount, rowCount);
+            }
+
+            // Verify numerical fields sum to expected value
+            for (String sumField : sumFields)
+            {
+                assertEquals("Field '" + sumField + "' does not sum to expected amount",
+                             sum.get(sumField).getValue().longValue(),
+                             dataset.groupBy().sum(sumField).first().getLong(0));
+            }
+
+            // Run SparkSQL checks
+            for (Consumer<Dataset<Row>> check : checks)
+            {
+                check.accept(dataset);
+            }
+
+            if (reset != null)
+            {
+                reset.run();
+            }
+        });
+    }
+
+    public static TestSchema.TestRow newUniqueRow(TestSchema schema, Map<String, TestSchema.TestRow> rows)
+    {
+        return newUniqueRow(schema::randomRow, rows);
+    }
+
+    public static TestSchema.TestRow newUniquePartitionDeletion(TestSchema schema, Map<String, TestSchema.TestRow> rows)
+    {
+        return newUniqueRow(schema::randomPartitionDelete, rows);
+    }
+
+    private static TestSchema.TestRow newUniqueRow(Supplier<TestSchema.TestRow> rowProvider,
+                                                   Map<String, TestSchema.TestRow> rows)
+    {
+        TestSchema.TestRow testRow;
+        do
+        {
+            testRow = rowProvider.get();
+        }
+        while (rows.containsKey(testRow.getKey()));  // Don't write duplicate rows
+        return testRow;
+    }
+
+    private void rotate(Object[] array, int shift)
+    {
+        ArrayUtils.reverse(array, 0, shift);
+        ArrayUtils.reverse(array, shift, array.length);
+        ArrayUtils.reverse(array, 0, array.length);
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/BulkSparkConfTest.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/BulkSparkConfTest.java
new file mode 100644
index 0000000..d2f7ded
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/BulkSparkConfTest.java
@@ -0,0 +1,218 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Maps;
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.cassandra.spark.bulkwriter.util.SbwKryoRegistrator;
+import org.apache.cassandra.spark.utils.BuildInfo;
+import org.apache.spark.SparkConf;
+import org.jetbrains.annotations.NotNull;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.isEmptyString;
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertTrue;
+
+public class BulkSparkConfTest
+{
+    private SparkConf sparkConf;
+    private BulkSparkConf bulkSparkConf;
+    private Map<String, String> defaultOptions;
+
+    @Before
+    public void before()
+    {
+        sparkConf = new SparkConf();
+        defaultOptions = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER);
+        defaultOptions.put(WriterOptions.SIDECAR_INSTANCES.name(), "127.0.0.1");
+        defaultOptions.put(WriterOptions.KEYSPACE.name(), "ks");
+        defaultOptions.put(WriterOptions.TABLE.name(), "table");
+        defaultOptions.put(WriterOptions.KEYSTORE_PASSWORD.name(), "dummy_password");
+        defaultOptions.put(WriterOptions.KEYSTORE_PATH.name(), "dummy_path");
+        bulkSparkConf = new BulkSparkConf(sparkConf, defaultOptions);
+    }
+
+    @Test
+    public void testGetBoolean()
+    {
+        sparkConf.set("spark.cassandra_analytics.job.skip_clean", "true");
+        assertThat(bulkSparkConf.getSkipClean(), is(true));
+    }
+
+    @Test
+    public void testGetLong()
+    {
+        sparkConf.set("spark.cassandra_analytics.sidecar.request.retries.delay.seconds", "2222");
+        assertThat(bulkSparkConf.getSidecarRequestRetryDelayInSeconds(), is(2222L));
+    }
+
+    @Test
+    public void testGetInt()
+    {
+        sparkConf.set("spark.cassandra_analytics.request.max_connections", "1234");
+        assertThat(bulkSparkConf.getMaxHttpConnections(), is(1234));
+    }
+
+    @Test
+    public void deprecatedSettingsAreHonored()
+    {
+        // Test that deprecated names of settings are in fact picked up correctly
+
+        sparkConf.set("qwerty.request.max_connections", "1234");
+        sparkConf.set("asdfgh.request.max_connections", "000000");
+        sparkConf.set("zxcvbn.request.response_timeout", "5678");
+        BulkSparkConf bulkSparkConf = new BulkSparkConf(sparkConf, defaultOptions)
+        {
+            @Override
+            @NotNull
+            protected List<String> getDeprecatedSettingPrefixes()
+            {
+                return ImmutableList.of("qwerty.", "asdfgh.", "zxcvbn.");
+            }
+        };
+
+        assertThat(bulkSparkConf.getMaxHttpConnections(), is(1234));
+        assertThat(bulkSparkConf.getHttpResponseTimeoutMs(), is(5678));
+    }
+
+    @Test
+    public void calculatesCoresCorrectlyForStaticAllocation()
+    {
+        sparkConf.set("spark.executor.cores", "5");
+        sparkConf.set("spark.executor.instances", "5");
+        assertThat(bulkSparkConf.getCores(), is(25));
+    }
+
+    @Test
+    public void calculatesCoresCorrectlyForDynamicAllocation()
+    {
+        sparkConf.set("spark.executor.cores", "6");
+        sparkConf.set("spark.dynamicAllocation.maxExecutors", "7");
+        assertThat(bulkSparkConf.getCores(), is(42));
+    }
+
+    @Test
+    public void ensureSetupSparkConfAddsPerformsNecessaryTasks()
+    {
+        assertThat(sparkConf.get("spark.kryo.registrator", ""), isEmptyString());
+        assertThat(sparkConf.get("spark.executor.extraJavaOptions", ""), isEmptyString());
+        BulkSparkConf.setupSparkConf(sparkConf, true);
+        assertEquals("," + SbwKryoRegistrator.class.getName(), sparkConf.get("spark.kryo.registrator", ""));
+        if (BuildInfo.isAtLeastJava11(BuildInfo.javaSpecificationVersion()))
+        {
+            assertEquals(BulkSparkConf.JDK11_OPTIONS, sparkConf.get("spark.executor.extraJavaOptions", ""));
+        }
+    }
+
+    @Test
+    public void withProperMtlsSettingsWillCreateSuccessfully()
+    {
+        // mTLS is now required, and the BulkSparkConf constructor fails if the options aren't present
+        Map<String, String> options = copyDefaultOptions();
+        SparkConf sparkConf = new SparkConf();
+        BulkSparkConf bulkSparkConf = new BulkSparkConf(sparkConf, options);
+    }
+
+    @Test
+    public void keystorePathRequiredIfBase64EncodedKeystoreNotSet()
+    {
+        Map<String, String> options = copyDefaultOptions();
+        options.remove(WriterOptions.KEYSTORE_PATH.name());
+        SparkConf sparkConf = new SparkConf();
+        NullPointerException npe = assertThrows(NullPointerException.class,
+                                                () -> new BulkSparkConf(sparkConf, options));
+        assertEquals("Keystore password was set. But both keystore path and base64 encoded string are not set. "
+                   + "Please either set option " + WriterOptions.KEYSTORE_PATH
+                   + " or option " + WriterOptions.KEYSTORE_BASE64_ENCODED, npe.getMessage());
+    }
+
+    @Test
+    public void testSkipClean()
+    {
+        assertFalse(bulkSparkConf.getSkipClean());
+        sparkConf.set(BulkSparkConf.SKIP_CLEAN, "true");
+        assertTrue(bulkSparkConf.getSkipClean());
+    }
+
+    @Test
+    public void testDefaultSidecarPort()
+    {
+        bulkSparkConf = new BulkSparkConf(new SparkConf(), defaultOptions);
+        assertEquals(9043, bulkSparkConf.getSidecarPort());
+    }
+
+    @Test
+    public void testSidecarPortSetByOptions()
+    {
+        Map<String, String> options = copyDefaultOptions();
+        options.put(WriterOptions.SIDECAR_PORT.name(), "9999");
+        bulkSparkConf = new BulkSparkConf(new SparkConf(), options);
+        assertEquals(9999, bulkSparkConf.getSidecarPort());
+    }
+
+    @Test
+    public void testSidecarPortSetByProperty()
+    {
+        // Spark conf loads values from system properties, but we can also test by calling `.set` explicitly.
+        // This makes the test not pollute global (System.properties) state but still tests the same basic path.
+        SparkConf conf = new SparkConf()
+                         .set(BulkSparkConf.SIDECAR_PORT, "9876");
+        bulkSparkConf = new BulkSparkConf(conf, defaultOptions);
+        assertEquals(9876, bulkSparkConf.getSidecarPort());
+    }
+
+    @Test
+    public void testKeystoreBase64EncodedStringSet()
+    {
+        Map<String, String> options = copyDefaultOptions();
+        options.remove(WriterOptions.KEYSTORE_PATH.name());
+        options.put(WriterOptions.KEYSTORE_BASE64_ENCODED.name(), "dummy_base64_encoded_keystore");
+        bulkSparkConf = new BulkSparkConf(sparkConf, defaultOptions);
+    }
+
+    @Test
+    public void testTrustStorePasswordSetPathNotSet()
+    {
+        Map<String, String> options = copyDefaultOptions();
+        options.put(WriterOptions.TRUSTSTORE_PATH.name(), "dummy");
+        NullPointerException npe = assertThrows(NullPointerException.class,
+                                                () -> new BulkSparkConf(sparkConf, options));
+        assertEquals("Trust Store Path was provided, but password is missing. "
+                   + "Please provide option " + WriterOptions.TRUSTSTORE_PASSWORD, npe.getMessage());
+    }
+
+    private Map<String, String> copyDefaultOptions()
+    {
+        TreeMap<String, String> map = new TreeMap<>(String.CASE_INSENSITIVE_ORDER);
+        map.putAll(defaultOptions);
+        return map;
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/CassandraRingMonitorTest.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/CassandraRingMonitorTest.java
new file mode 100644
index 0000000..a06e217
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/CassandraRingMonitorTest.java
@@ -0,0 +1,137 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+import java.util.concurrent.TimeUnit;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.cassandra.spark.bulkwriter.token.CassandraRing;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class CassandraRingMonitorTest
+{
+    private int changeCount = 0;
+    private CassandraRing<RingInstance> ring;
+    private MockScheduledExecutorService executorService;
+
+    @Before
+    public void setup()
+    {
+        changeCount = 0;
+        ring = buildRing(0);
+        executorService = new MockScheduledExecutorService();
+    }
+
+    @Test
+    public void noRingChange() throws Exception
+    {
+        MockScheduledExecutorService executorService = new MockScheduledExecutorService();
+        CassandraRingMonitor crm = new CassandraRingMonitor(() -> ring,
+                                                            event -> changeCount++,
+                                                            1,
+                                                            TimeUnit.SECONDS,
+                                                            executorService);
+        // Make no changes to the ring and call again
+        executorService.runCommand();
+        assertEquals(0, changeCount);
+        assertFalse(crm.getRingChanged());
+}
+
+    @Test
+    public void ringChanged() throws Exception
+    {
+        CassandraRingMonitor crm = new CassandraRingMonitor(() -> ring,
+                                                            event -> changeCount++,
+                                                            1,
+                                                            TimeUnit.SECONDS,
+                                                            executorService);
+        // Make no changes to the ring and call again
+        executorService.runCommand();
+        assertEquals(0, changeCount);
+        assertFalse(crm.getRingChanged());
+        // Change the ring
+        ring = buildRing(1);
+        // Run the ring check again
+        executorService.runCommand();
+        assertEquals(1, changeCount);
+        assertTrue(crm.getRingChanged());
+        assertTrue(executorService.isStopped());
+    }
+
+    @Test
+    public void stopShutsDownExecutor() throws Exception
+    {
+        CassandraRingMonitor crm = new CassandraRingMonitor(() -> ring,
+                                                            event -> changeCount++,
+                                                            1,
+                                                            TimeUnit.SECONDS,
+                                                            executorService);
+        crm.stop();
+        assertTrue(executorService.isStopped());
+    }
+
+    @Test
+    public void passesTimeUnitCorrectly() throws Exception
+    {
+        CassandraRingMonitor crm = new CassandraRingMonitor(() -> ring,
+                                                            event -> changeCount++,
+                                                            10,
+                                                            TimeUnit.HOURS,
+                                                            executorService);
+        crm.stop();
+        assertEquals(10, executorService.getPeriod());
+        assertEquals(TimeUnit.HOURS, executorService.getTimeUnit());
+    }
+
+    @Test
+    public void multipleRingChangedFireOnceFalse() throws Exception
+    {
+        CassandraRingMonitor crm = new CassandraRingMonitor(() -> ring,
+                                                            event -> changeCount++,
+                                                            1,
+                                                            TimeUnit.SECONDS,
+                                                            executorService);
+        // Make no changes to the ring and call again
+        executorService.runCommand();
+        assertEquals(0, changeCount);
+        assertFalse(crm.getRingChanged());
+        // Change the ring
+        ring = buildRing(1);
+        // Run the ring check again & make sure we increment change count
+        executorService.runCommand();
+        assertEquals(1, changeCount);
+        assertTrue(crm.getRingChanged());
+        // Make sure another check does not fire again, reset local state and change ring again
+        ring = buildRing(2);
+        executorService.runCommand();
+        assertEquals(2, changeCount);
+        assertTrue(crm.getRingChanged());
+    }
+
+    private CassandraRing<RingInstance> buildRing(int initialToken)
+    {
+        return RingUtils.buildRing(initialToken, "dev3", "dev3", "DEV", "test", 3);
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/CqlTableInfoProviderTest.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/CqlTableInfoProviderTest.java
new file mode 100644
index 0000000..232f7a5
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/CqlTableInfoProviderTest.java
@@ -0,0 +1,127 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+import org.junit.Test;
+
+import static org.apache.cassandra.spark.bulkwriter.CqlTableInfoProvider.removeDeprecatedOptions;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThrows;
+
+public class CqlTableInfoProviderTest
+{
+    @Test
+    public void testRemoveDeprecatedOptionsInvalidInput()
+    {
+        assertThrows(NullPointerException.class, () -> removeDeprecatedOptions(null));
+        assertEquals("", removeDeprecatedOptions(""));
+        assertEquals("qwerty", removeDeprecatedOptions("qwerty"));
+    }
+
+    @Test
+    public void testRemoveDeprecatedOptionsOptionNames()
+    {
+        assertEquals("... WITH qwerty = 42 ...",
+                     removeDeprecatedOptions("... WITH qwerty = 42 ..."));
+        assertEquals("... WITH ...",
+                     removeDeprecatedOptions("... WITH read_repair_chance = 42 ..."));
+        assertEquals("... WITH ...",
+                     removeDeprecatedOptions("... WITH dclocal_read_repair_chance = 42 ..."));
+        assertEquals("... WITH dclocal_dclocal_read_repair_chance = 42 ...",
+                     removeDeprecatedOptions("... WITH dclocal_dclocal_read_repair_chance = 42 ..."));
+    }
+
+    @Test
+    public void testRemoveDeprecatedOptionsOptionValues()
+    {
+        assertEquals("... WITH ...",
+                     removeDeprecatedOptions("... WITH read_repair_chance = -42 ..."));
+        assertEquals("... WITH ...",
+                     removeDeprecatedOptions("... WITH read_repair_chance = 420.0e-1 ..."));
+        assertEquals("... WITH ...",
+                     removeDeprecatedOptions("... WITH read_repair_chance = +.42E+1.0 ..."));
+        assertEquals("... WITH read_repair_chance = true ...",
+                     removeDeprecatedOptions("... WITH read_repair_chance = true ..."));
+    }
+
+    @Test
+    public void testRemoveDeprecatedOptionsOptionsOrder()
+    {
+        assertEquals("... WITH ...",
+                     removeDeprecatedOptions("... WITH read_repair_chance = 1 ..."));
+        assertEquals("... WITH qwerty = 42 ...",
+                     removeDeprecatedOptions("... WITH read_repair_chance = 1 AND qwerty = 42 ..."));
+        assertEquals("... WITH qwerty = 42 ...",
+                     removeDeprecatedOptions("... WITH qwerty = 42 AND read_repair_chance = 1 ..."));
+        assertEquals("... WITH qwerty = 42 ...",
+                     removeDeprecatedOptions("... WITH read_repair_chance = 1 AND qwerty = 42"
+                                           + " AND dclocal_read_repair_chance = 1 ..."));
+        assertEquals("... WITH qwerty = 42 AND asdfgh = 43 ...",
+                     removeDeprecatedOptions("... WITH qwerty = 42 AND read_repair_chance = 1 AND asdfgh = 43 ..."));
+        assertEquals("... WITH qwerty = 42 AND asdfgh = 43 AND zxcvbn = 44 ...",
+                     removeDeprecatedOptions("... WITH qwerty = 42 AND read_repair_chance = 1 AND asdfgh = 43"
+                                           + " AND dclocal_read_repair_chance = 1 AND zxcvbn = 44 ..."));
+        assertEquals("... WITH qwerty = 42 AND asdfgh = 43 AND zxcvbn = 44 ...",
+                     removeDeprecatedOptions("... WITH qwerty = 42 AND asdfgh = 43 AND zxcvbn = 44 ..."));
+    }
+
+    @Test
+    public void testRemoveDeprecatedOptionsStatementCase()
+    {
+        assertEquals("... WITH ...",
+                     removeDeprecatedOptions("... WITH read_repair_chance = 1 AND dclocal_read_repair_chance = 1 ..."));
+        assertEquals("... WITH ...",
+                     removeDeprecatedOptions("... WITH READ_REPAIR_CHANCE = 1 AND DCLOCAL_READ_REPAIR_CHANCE = 1 ..."));
+        assertEquals("... with ...",
+                     removeDeprecatedOptions("... with read_repair_chance = 1 and dclocal_read_repair_chance = 1 ..."));
+        assertEquals("... WiTh ...",
+                     removeDeprecatedOptions("... WiTh ReAd_RePaIr_ChAnCe = 1 AnD dClOcAl_ReAd_RePaIr_ChAnCe = 1 ..."));
+    }
+
+    @Test
+    public void testRemoveDeprecatedOptionsRealStatement()
+    {
+        String cql = "CREATE TABLE test_simple_rf_3_batch_10000_splits_3.test"
+                   + " (id int, course text, foo text, marks int, PRIMARY KEY ((id, course)))"
+                   + " WITH read_repair_chance = 0.0 AND dclocal_read_repair_chance = 0.0"
+                   + " AND gc_grace_seconds = 864000 AND bloom_filter_fp_chance = 0.1"
+                   + " AND caching = { 'keys' : 'ALL', 'rows_per_partition' : 'NONE' } AND comment = ''"
+                   + " AND compaction = { 'class' : 'org.apache.cassandra.db.compaction.LeveledCompactionStrategy',"
+                   +                    " 'enabled' : 'true', 'max_threshold' : 32, 'min_threshold' : 4 }"
+                   + " AND compression = { 'chunk_length_in_kb' : 16,"
+                   +                     " 'class' : 'org.apache.cassandra.io.compress.ZstdCompressor' }"
+                   + " AND default_time_to_live = 0 AND speculative_retry = '99p' AND min_index_interval = 128"
+                   + " AND max_index_interval = 2048 AND crc_check_chance = 1.0 AND cdc = false"
+                   + " AND memtable_flush_period_in_ms = 0;";
+        String expected = "CREATE TABLE test_simple_rf_3_batch_10000_splits_3.test"
+                        + " (id int, course text, foo text, marks int, PRIMARY KEY ((id, course)))"
+                        + " WITH gc_grace_seconds = 864000 AND bloom_filter_fp_chance = 0.1"
+                        + " AND caching = { 'keys' : 'ALL', 'rows_per_partition' : 'NONE' } AND comment = ''"
+                        + " AND compaction = { 'class' : 'org.apache.cassandra.db.compaction.LeveledCompactionStrategy',"
+                        +                    " 'enabled' : 'true', 'max_threshold' : 32, 'min_threshold' : 4 }"
+                        + " AND compression = { 'chunk_length_in_kb' : 16,"
+                        +                     " 'class' : 'org.apache.cassandra.io.compress.ZstdCompressor' }"
+                        + " AND default_time_to_live = 0 AND speculative_retry = '99p' AND min_index_interval = 128"
+                        + " AND max_index_interval = 2048 AND crc_check_chance = 1.0 AND cdc = false"
+                        + " AND memtable_flush_period_in_ms = 0;";
+        String actual = removeDeprecatedOptions(cql);
+        assertEquals(expected, actual);
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/DecoratedKeyTest.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/DecoratedKeyTest.java
new file mode 100644
index 0000000..e621d00
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/DecoratedKeyTest.java
@@ -0,0 +1,63 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+
+public class DecoratedKeyTest
+{
+    @Test
+    public void compareToBytesDifferent()
+    {
+        DecoratedKey key1 = new DecoratedKey(BigInteger.valueOf(1), ByteBuffer.wrap(new byte[]{'a', 'a'}));
+        DecoratedKey key2 = new DecoratedKey(BigInteger.valueOf(1), ByteBuffer.wrap(new byte[]{'a', 'z'}));
+        assertNotEquals(key2.compareTo(key1), 0);
+    }
+
+    @Test
+    public void compareToBytesDifferentLengths()
+    {
+        DecoratedKey key1 = new DecoratedKey(BigInteger.valueOf(1), ByteBuffer.wrap(new byte[]{'a'}));
+        DecoratedKey key2 = new DecoratedKey(BigInteger.valueOf(1), ByteBuffer.wrap(new byte[]{'a', 'a'}));
+        assertNotEquals(key2.compareTo(key1), 0);
+    }
+
+    @Test
+    public void compareToTokensDifferent()
+    {
+        DecoratedKey key1 = new DecoratedKey(BigInteger.valueOf(1), ByteBuffer.wrap(new byte[]{'a'}));
+        DecoratedKey key2 = new DecoratedKey(BigInteger.valueOf(2), ByteBuffer.wrap(new byte[]{'a'}));
+        assertNotEquals(key1.compareTo(key2), 0);
+    }
+
+    @Test
+    public void equals()
+    {
+        DecoratedKey key1 = new DecoratedKey(BigInteger.valueOf(1), ByteBuffer.wrap(new byte[]{'a'}));
+        DecoratedKey key2 = new DecoratedKey(BigInteger.valueOf(1), ByteBuffer.wrap(new byte[]{'a'}));
+        assertEquals(key1, key2);
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/MockBulkWriterContext.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/MockBulkWriterContext.java
new file mode 100644
index 0000000..262b7df
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/MockBulkWriterContext.java
@@ -0,0 +1,405 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.BiFunction;
+import java.util.function.Predicate;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.commons.lang3.tuple.Pair;
+
+import org.apache.cassandra.sidecar.common.data.TimeSkewResponse;
+import org.apache.cassandra.spark.bulkwriter.token.CassandraRing;
+import org.apache.cassandra.spark.bulkwriter.token.ConsistencyLevel;
+import org.apache.cassandra.spark.common.MD5Hash;
+import org.apache.cassandra.spark.common.client.ClientException;
+import org.apache.cassandra.spark.common.client.InstanceState;
+import org.apache.cassandra.spark.common.model.BulkFeatures;
+import org.apache.cassandra.spark.common.model.CassandraInstance;
+import org.apache.cassandra.spark.common.schema.ColumnType;
+import org.apache.cassandra.spark.common.schema.ColumnTypes;
+import org.apache.cassandra.spark.data.CqlField;
+import org.apache.cassandra.spark.data.partitioner.Partitioner;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.StructType;
+
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.DATE;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.INT;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.VARCHAR;
+import static org.apache.cassandra.spark.bulkwriter.TableSchemaTestCommon.mockCqlType;
+
+public class MockBulkWriterContext implements BulkWriterContext, ClusterInfo, JobInfo, SchemaInfo, DataTransferApi
+{
+    private static final long serialVersionUID = -2912371629236770646L;
+    private RowBufferMode rowBufferMode = RowBufferMode.UNBUFFERRED;
+    private ConsistencyLevel.CL consistencyLevel;
+
+    public interface CommitResultSupplier extends BiFunction<List<String>, String, RemoteCommitResult>
+    {
+    }
+
+    public static final String DEFAULT_CASSANDRA_VERSION = "cassandra-4.0.2";
+
+    private final UUID jobId;
+    private Supplier<Long> timeProvider = System::currentTimeMillis;
+    private boolean skipClean = false;
+    public int refreshClusterInfoCallCount = 0;  // CHECKSTYLE IGNORE: Public mutable field
+    private final Map<CassandraInstance, List<UploadRequest>> uploads = new HashMap<>();
+    private final Map<CassandraInstance, List<String>> commits = new HashMap<>();
+    final Pair<StructType, ImmutableMap<String, CqlField.CqlType>> validPair;
+    private final TableSchema schema;
+    private final Set<CassandraInstance> cleanCalledForInstance = new HashSet<>();
+    private boolean instancesAreAvailable = true;
+    private boolean cleanShouldThrow = false;
+    private final CassandraRing<RingInstance> ring;
+    private final TokenPartitioner tokenPartitioner;
+    private final String cassandraVersion;
+    private CommitResultSupplier crSupplier = (uuids, dc) -> new RemoteCommitResult(true, Collections.emptyList(), uuids, null);
+
+    private Predicate<CassandraInstance> uploadRequestConsumer = instance -> true;
+
+    public MockBulkWriterContext(CassandraRing<RingInstance> ring,
+                                 String cassandraVersion,
+                                 ConsistencyLevel.CL consistencyLevel)
+    {
+        this.ring = ring;
+        this.tokenPartitioner = new TokenPartitioner(ring, 1, 2, 2, false);
+        this.cassandraVersion = cassandraVersion;
+        this.consistencyLevel = consistencyLevel;
+        validPair = TableSchemaTestCommon.buildMatchedDataframeAndCqlColumns(
+                new String[]{"id", "date", "course", "marks"},
+                new org.apache.spark.sql.types.DataType[]{DataTypes.IntegerType, DataTypes.DateType, DataTypes.StringType, DataTypes.IntegerType},
+                new CqlField.CqlType[]{mockCqlType(INT), mockCqlType(DATE), mockCqlType(VARCHAR), mockCqlType(INT)});
+        StructType validDataFrameSchema = validPair.getKey();
+        ImmutableMap<String, CqlField.CqlType> validCqlColumns = validPair.getValue();
+        String[] partitionKeyColumns = {"id", "date"};
+        String[] primaryKeyColumnNames = {"id", "date"};
+        ColumnType<?>[] partitionKeyColumnTypes = {ColumnTypes.INT, ColumnTypes.INT};
+        this.schema = new TableSchemaTestCommon.MockTableSchemaBuilder()
+                      .withCqlColumns(validCqlColumns)
+                      .withPartitionKeyColumns(partitionKeyColumns)
+                      .withPrimaryKeyColumnNames(primaryKeyColumnNames)
+                      .withCassandraVersion(cassandraVersion)
+                      .withPartitionKeyColumnTypes(partitionKeyColumnTypes)
+                      .withWriteMode(WriteMode.INSERT)
+                      .withDataFrameSchema(validDataFrameSchema)
+                      .build();
+        this.jobId = java.util.UUID.randomUUID();
+    }
+
+    public Supplier<Long> getTimeProvider()
+    {
+        return timeProvider;
+    }
+
+    public void setTimeProvider(Supplier<Long> timeProvider)
+    {
+        this.timeProvider = timeProvider;
+    }
+
+    public MockBulkWriterContext(CassandraRing<RingInstance> ring)
+    {
+        this(ring, DEFAULT_CASSANDRA_VERSION, ConsistencyLevel.CL.LOCAL_QUORUM);
+    }
+
+    @Override
+    public void shutdown()
+    {
+    }
+
+    @Override
+    public TimeSkewResponse getTimeSkew(List<RingInstance> replicas)
+    {
+        return new TimeSkewResponse(timeProvider.get(), 60);
+    }
+
+    @Override
+    public String getKeyspaceSchema(boolean cached)
+    {
+        // TODO: Fix me
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void refreshClusterInfo()
+    {
+        refreshClusterInfoCallCount++;
+    }
+
+    @Override
+    public ConsistencyLevel.CL getConsistencyLevel()
+    {
+        return consistencyLevel;
+    }
+
+    public void setConsistencyLevel(ConsistencyLevel.CL consistencyLevel)
+    {
+        this.consistencyLevel = consistencyLevel;
+    }
+
+    @Override
+    public String getLocalDC()
+    {
+        return "DC1";
+    }
+
+    @Override
+    public RowBufferMode getRowBufferMode()
+    {
+        return rowBufferMode;
+    }
+
+    public void setRowBufferMode(RowBufferMode rowBufferMode)
+    {
+        this.rowBufferMode = rowBufferMode;
+    }
+
+    @Override
+    public int getSstableDataSizeInMB()
+    {
+        return 128;
+    }
+
+    @Override
+    public int getSstableBatchSize()
+    {
+        return 1;
+    }
+
+    @Override
+    public int getCommitBatchSize()
+    {
+        return 1;
+    }
+
+    @Override
+    public boolean validateSSTables()
+    {
+        return true;
+    }
+
+    @Override
+    public boolean skipExtendedVerify()
+    {
+        return false;
+    }
+
+    @Override
+    public boolean getSkipClean()
+    {
+        return skipClean;
+    }
+
+    public void setSkipCleanOnFailures(boolean skipClean)
+    {
+        this.skipClean = skipClean;
+    }
+
+    @Override
+    public int getCommitThreadsPerInstance()
+    {
+        return 1;
+    }
+
+    @Override
+    public CassandraRing<RingInstance> getRing(boolean cached)
+    {
+        return ring;
+    }
+
+    @Override
+    public UUID getId()
+    {
+        return jobId;
+    }
+
+    @Override
+    public TokenPartitioner getTokenPartitioner()
+    {
+        return tokenPartitioner;
+    }
+
+    @Override
+    public TableSchema getTableSchema()
+    {
+        return schema;
+    }
+
+    @Override
+    public Partitioner getPartitioner()
+    {
+        return Partitioner.Murmur3Partitioner;
+    }
+
+    @Override
+    public void checkBulkWriterIsEnabledOrThrow()
+    {
+        throw new RuntimeException(String.format("Aborting Bulk Writer! feature %s is disabled for cluster",
+                                                 BulkFeatures.BULK_WRITER));
+    }
+
+    @Override
+    public String getLowestCassandraVersion()
+    {
+        return cassandraVersion;
+    }
+
+    @Override
+    public RemoteCommitResult commitSSTables(CassandraInstance instance, String migrationId, List<String> uuids)
+    {
+        commits.compute(instance, (ignored, commitList) -> {
+            if (commitList == null)
+            {
+                commitList = new ArrayList<>();
+            }
+            commitList.add(migrationId);
+            return commitList;
+        });
+        return crSupplier.apply(buildCompleteBatchIds(uuids), instance.getDataCenter());
+    }
+
+    private List<String> buildCompleteBatchIds(List<String> uuids)
+    {
+        return uuids.stream().map(uuid -> uuid + "-" + jobId).collect(Collectors.toList());
+    }
+
+    @Override
+    public void cleanUploadSession(CassandraInstance instance, String sessionID, String jobID) throws ClientException
+    {
+        cleanCalledForInstance.add(instance);
+        if (cleanShouldThrow)
+        {
+            throw new ClientException("Clean was called but was set to throw");
+        }
+    }
+
+    @Override
+    public void uploadSSTableComponent(Path componentFile,
+                                       int ssTableIdx,
+                                       CassandraInstance instance,
+                                       String sessionID,
+                                       MD5Hash fileHash) throws ClientException
+    {
+        boolean uploadSucceeded = uploadRequestConsumer.test(instance);
+        uploads.compute(instance, (k, pathList) -> {
+            if (pathList == null)
+            {
+                pathList = new ArrayList<>();
+            }
+            pathList.add(new UploadRequest(componentFile, ssTableIdx, instance, sessionID, fileHash, uploadSucceeded));
+            return pathList;
+        });
+        if (!uploadSucceeded)
+        {
+            throw new ClientException("Failed upload");
+        }
+    }
+
+    @Override
+    public Map<RingInstance, InstanceAvailability> getInstanceAvailability()
+    {
+        return null;
+    }
+
+    @Override
+    public boolean instanceIsAvailable(RingInstance ringInstance)
+    {
+        return instancesAreAvailable;
+    }
+
+    @Override
+    public InstanceState getInstanceState(RingInstance ringInstance)
+    {
+        return InstanceState.NORMAL;
+    }
+
+    public void setUploadSupplier(Predicate<CassandraInstance> uploadRequestConsumer)
+    {
+        this.uploadRequestConsumer = uploadRequestConsumer;
+    }
+
+    public void setInstancesAreAvailable(boolean instancesAreAvailable)
+    {
+        this.instancesAreAvailable = instancesAreAvailable;
+    }
+
+    public List<CassandraInstance> getCleanedInstances()
+    {
+        return new ArrayList<>(cleanCalledForInstance);
+    }
+
+    public void setCleanShouldThrow(boolean cleanShouldThrow)
+    {
+        this.cleanShouldThrow = cleanShouldThrow;
+    }
+
+    public Map<CassandraInstance, List<UploadRequest>> getUploads()
+    {
+        return uploads;
+    }
+
+    public CommitResultSupplier setCommitResultSupplier(CommitResultSupplier supplier)
+    {
+        CommitResultSupplier oldSupplier = crSupplier;
+        crSupplier = supplier;
+        return oldSupplier;
+    }
+
+    @Override
+    public ClusterInfo cluster()
+    {
+        return this;
+    }
+
+    @Override
+    public JobInfo job()
+    {
+        return this;
+    }
+
+    @Override
+    public SchemaInfo schema()
+    {
+        return this;
+    }
+
+    @Override
+    public DataTransferApi transfer()
+    {
+        return this;
+    }
+
+    @Override
+    public String getFullTableName()
+    {
+        return "keyspace.table";
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/MockScheduledExecutorService.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/MockScheduledExecutorService.java
new file mode 100644
index 0000000..41c6d75
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/MockScheduledExecutorService.java
@@ -0,0 +1,182 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.Delayed;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import org.jetbrains.annotations.NotNull;
+
+public class MockScheduledExecutorService extends ScheduledThreadPoolExecutor
+{
+    private Runnable command;
+    private boolean stopped = false;
+    private long period;
+    private TimeUnit timeUnit;
+    List<MockScheduledFuture<?>> futures = new ArrayList<>();
+
+    public MockScheduledExecutorService()
+    {
+        super(0);
+    }
+
+    @NotNull
+    @Override
+    public ScheduledFuture<?> scheduleAtFixedRate(@NotNull Runnable command,
+                                                  long initialDelay,
+                                                  long period,
+                                                  @NotNull TimeUnit unit)
+    {
+        this.period = period;
+        this.timeUnit = unit;
+        this.command = command;
+        return new MockScheduledFuture<Void>(command);
+    }
+
+    @Override
+    public ScheduledFuture<?> submit(Runnable command)
+    {
+        MockScheduledFuture<?> future = new MockScheduledFuture<>(command);
+        futures.add(future);
+        return future;
+    }
+
+    @NotNull
+    @Override
+    public List<Runnable> shutdownNow()
+    {
+        stopped = true;
+        return new ArrayList<>();
+    }
+
+    @Override
+    public void shutdown()
+    {
+        stopped = true;
+    }
+
+    public void assertFuturesCalled()
+    {
+        futures.forEach(MockScheduledFuture::assertGetCalled);
+    }
+
+    public void runCommand()
+    {
+        command.run();
+    }
+
+    public boolean isStopped()
+    {
+        return stopped;
+    }
+
+    public long getPeriod()
+    {
+        return period;
+    }
+
+    public TimeUnit getTimeUnit()
+    {
+        return timeUnit;
+    }
+
+    static class MockScheduledFuture<V> implements ScheduledFuture<V>
+    {
+        private final Callable<V> command;
+        private boolean getCalled = false;
+
+        MockScheduledFuture(Callable<V> command)
+        {
+            this.command = command;
+        }
+
+        MockScheduledFuture(Runnable command)
+        {
+            this.command = () -> {
+                command.run();
+                return null;
+            };
+        }
+
+        @Override
+        public long getDelay(@NotNull TimeUnit unit)
+        {
+            return 0;
+        }
+
+        @Override
+        public int compareTo(@NotNull Delayed that)
+        {
+            return 0;
+        }
+
+        @Override
+        public boolean cancel(boolean mayInterruptIfRunning)
+        {
+            return false;
+        }
+
+        @Override
+        public boolean isCancelled()
+        {
+            return false;
+        }
+
+        @Override
+        public boolean isDone()
+        {
+            return false;
+        }
+
+        @Override
+        public V get() throws InterruptedException, ExecutionException
+        {
+            getCalled = true;
+            try
+            {
+                return command.call();
+            }
+            catch (Exception exception)
+            {
+                throw new ExecutionException(exception);
+            }
+        }
+
+        @Override
+        public V get(long timeout,
+                     @NotNull TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException
+        {
+            getCalled = true;
+            return null;
+        }
+
+        public void assertGetCalled()
+        {
+            assert getCalled;
+        }
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/MockTableWriter.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/MockTableWriter.java
new file mode 100644
index 0000000..4003f54
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/MockTableWriter.java
@@ -0,0 +1,102 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.commons.io.FileUtils;
+
+import org.apache.cassandra.bridge.SSTableWriter;
+
+public class MockTableWriter implements SSTableWriter
+{
+    private static final String BASE_NAME = "/test-test-me-1-big-";
+    private static final String[] TABLE_COMPONENTS = {"Data.db",
+                                                      "Index.db",
+                                                      "Filter.db",
+                                                      "CompressionInfo.db",
+                                                      "Statistics.db",
+                                                      "Digest.crc32",
+                                                      "Summary.db",
+                                                      "TOC.txt"};
+
+    private Path outDir;
+    private boolean addRowThrows;
+    private final ArrayList<Object[]> rows = new ArrayList<>();
+    private boolean closed = false;
+
+    public MockTableWriter(Path outDir)
+    {
+        this.outDir = outDir;
+    }
+
+    public MockTableWriter setOutDir(Path outDir)
+    {
+        this.outDir = outDir;
+        return this;
+    }
+
+    public void setAddRowThrows(boolean addRowThrows)
+    {
+        this.addRowThrows = addRowThrows;
+    }
+
+    @Override
+    public void addRow(Object... values)
+    {
+        if (addRowThrows)
+        {
+            throw new RuntimeException("Failed to write because addRow throws");
+        }
+        rows.add(values);
+    }
+
+    @Override
+    public void close() throws IOException
+    {
+        // Create files to mimic SSTableWriter
+        // TODO: Instead, we shouldn't have SSTableWriter return the outDir - we should
+        //       provide a way to iterate over the data files and pass a callable of some kind in
+        for (String component: TABLE_COMPONENTS)
+        {
+            Path path = Paths.get(outDir.toString(), BASE_NAME + component);
+            System.out.format("Writing mock component %s\n", path);
+            Files.createFile(path);
+        }
+        closed = true;
+    }
+
+    @VisibleForTesting
+    public Path getOutDir()
+    {
+        return outDir;
+    }
+
+    @VisibleForTesting
+    public void removeOutDir() throws IOException
+    {
+        FileUtils.deleteDirectory(outDir.toFile());
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/NonValidatingTestSSTableWriter.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/NonValidatingTestSSTableWriter.java
new file mode 100644
index 0000000..1d1c167
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/NonValidatingTestSSTableWriter.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.cassandra.spark.bulkwriter;
+
+import java.nio.file.Path;
+
+class NonValidatingTestSSTableWriter extends SSTableWriter
+{
+    NonValidatingTestSSTableWriter(MockTableWriter tableWriter, Path path)
+    {
+        super(tableWriter, path);
+    }
+
+    @Override
+    public void validateSSTables(BulkWriterContext writerContext, int partitionId)
+    {
+        // Skip validation for these tests
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/RecordWriterTest.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/RecordWriterTest.java
new file mode 100644
index 0000000..4e60ec6
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/RecordWriterTest.java
@@ -0,0 +1,159 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+import java.math.BigInteger;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+
+import com.google.common.collect.Range;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.rules.TemporaryFolder;
+
+import org.apache.cassandra.spark.bulkwriter.token.CassandraRing;
+import org.apache.cassandra.spark.common.model.CassandraInstance;
+import scala.Tuple2;
+
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertThat;
+
+public class RecordWriterTest
+{
+    public static final int REPLICA_COUNT = 3;
+    public static final int FILES_PER_SSTABLE = 8;
+    public static final int UPLOADED_TABLES = 3;
+    @Rule
+    public TemporaryFolder folder = new TemporaryFolder();
+
+    @Rule
+    public ExpectedException exception = ExpectedException.none();
+    private CassandraRing<RingInstance> ring;
+    private RecordWriter rw;
+    private MockTableWriter tw;
+    private Tokenizer tokenizer;
+    private Range<BigInteger> range;
+    private MockBulkWriterContext writerContext;
+    private TestTaskContext tc;
+
+    @Before
+    public void setUp()
+    {
+        tw = new MockTableWriter(folder.getRoot().toPath());
+        ring = RingUtils.buildRing(0, "app", "cluster", "DC1", "test", 12);
+        writerContext = new MockBulkWriterContext(ring);
+        tc = new TestTaskContext();
+        range = writerContext.job().getTokenPartitioner().getTokenRange(tc.partitionId());
+        tokenizer = new Tokenizer(writerContext);
+    }
+
+    @Test
+    public void testSuccessfulWrite()
+    {
+        rw = new RecordWriter(writerContext, () -> tc, SSTableWriter::new);
+        Iterator<Tuple2<DecoratedKey, Object[]>> data = generateData(5, true);
+        rw.write(data);
+        Map<CassandraInstance, List<UploadRequest>> uploads = writerContext.getUploads();
+        assertThat(uploads.keySet().size(), is(REPLICA_COUNT));  // Should upload to 3 replicas
+        assertThat(uploads.values().stream().mapToInt(List::size).sum(), is(REPLICA_COUNT * FILES_PER_SSTABLE * UPLOADED_TABLES));
+        List<UploadRequest> requests = uploads.values().stream().flatMap(List::stream).collect(Collectors.toList());
+        for (UploadRequest ur: requests)
+        {
+            assertNotNull(ur.fileHash);
+        }
+    }
+
+    @Test
+    public void testCorruptSSTable()
+    {
+        // TODO: Add better error handling with human-readable exception messages in SSTableReader::new
+        exception.expect(RuntimeException.class);
+        rw = new RecordWriter(writerContext, () -> tc, (wc, path) ->  new SSTableWriter(tw.setOutDir(path), path));
+        Iterator<Tuple2<DecoratedKey, Object[]>> data = generateData(5, true);
+        rw.write(data);
+        Map<CassandraInstance, List<UploadRequest>> uploads = writerContext.getUploads();
+        assertThat(uploads.keySet().size(), is(REPLICA_COUNT));  // Should upload to 3 replicas
+        assertThat(uploads.values().stream().mapToInt(List::size).sum(), is(REPLICA_COUNT * FILES_PER_SSTABLE * UPLOADED_TABLES));
+    }
+
+    @Test
+    public void testWriteWithOutOfRangeTokenFails()
+    {
+        rw = new RecordWriter(writerContext, () -> tc, (wc, path) -> new SSTableWriter(tw, folder.getRoot().toPath()));
+        exception.expectMessage("Received Token 5765203080415074583 outside of expected range [-9223372036854775808‥0]");
+        Iterator<Tuple2<DecoratedKey, Object[]>> data = generateData(5, false);
+        rw.write(data);
+    }
+
+    @Test
+    public void testAddRowThrowingFails()
+    {
+        rw = new RecordWriter(writerContext, () -> tc, (wc, path) -> new SSTableWriter(tw, folder.getRoot().toPath()));
+        tw.setAddRowThrows(true);
+        exception.expectMessage("java.lang.RuntimeException: Failed to write because addRow throws");
+        Iterator<Tuple2<DecoratedKey, Object[]>> data = generateData(5, true);
+        rw.write(data);
+    }
+
+    @Test
+    public void testBadTimeSkewFails()
+    {
+        // Mock context returns a 60-minute allowable time skew, so we use something just outside the limits
+        long sixtyOneMinutesInMillis = TimeUnit.MINUTES.toMillis(61);
+        rw = new RecordWriter(writerContext, () -> tc, (wc, path) -> new SSTableWriter(tw, folder.getRoot().toPath()));
+        writerContext.setTimeProvider(() -> System.currentTimeMillis() - sixtyOneMinutesInMillis);
+        exception.expectMessage("Time skew between Spark and Cassandra is too large. Allowable skew is 60 minutes. Spark executor time is ");
+        Iterator<Tuple2<DecoratedKey, Object[]>> data = generateData(5, true);
+        rw.write(data);
+    }
+
+    @Test
+    public void testTimeSkewWithinLimitsSucceeds()
+    {
+        // Mock context returns a 60-minute allowable time skew, so we use something just inside the limits
+        long fiftyNineMinutesInMillis = TimeUnit.MINUTES.toMillis(59);
+        long remoteTime = System.currentTimeMillis() - fiftyNineMinutesInMillis;
+        rw = new RecordWriter(writerContext, () -> tc, SSTableWriter::new);
+        writerContext.setTimeProvider(() -> remoteTime);  // Return a very low "current time" to make sure we fail if skew is too bad
+        Iterator<Tuple2<DecoratedKey, Object[]>> data = generateData(5, true);
+        rw.write(data);
+    }
+
+    private Iterator<Tuple2<DecoratedKey, Object[]>> generateData(int numValues, boolean onlyInRange)
+    {
+        Stream<Tuple2<DecoratedKey, Object[]>> source = IntStream.iterate(0, integer -> integer + 1).mapToObj(index -> {
+            Object[] columns = {index, index, "foo" + index, index};
+            return Tuple2.apply(tokenizer.getDecoratedKey(columns), columns);
+        });
+        if (onlyInRange)
+        {
+            source = source.filter(val -> range.contains(val._1.getToken()));
+        }
+        return source.limit(numValues).iterator();
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/RingInstanceSerializationTest.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/RingInstanceSerializationTest.java
new file mode 100644
index 0000000..84f8f46
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/RingInstanceSerializationTest.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.cassandra.spark.bulkwriter;
+
+import org.junit.Test;
+
+import org.apache.cassandra.sidecar.common.data.RingEntry;
+
+import static org.apache.cassandra.spark.utils.SerializationUtils.deserialize;
+import static org.apache.cassandra.spark.utils.SerializationUtils.serialize;
+import static org.junit.Assert.assertEquals;
+
+public class RingInstanceSerializationTest
+{
+    @Test
+    public void testRingSerializesCorrectly()
+    {
+        int dcOffset = 0;
+        String dataCenter = "DC1";
+        int index = 0;
+        int initialToken = 0;
+        RingInstance ring = new RingInstance(new RingEntry.Builder()
+                                             .address("127.0." + dcOffset + "." + index)
+                                             .port(7000)
+                                             .datacenter(dataCenter)
+                                             .load("0")
+                                             .token(Integer.toString(initialToken + dcOffset + 100 * index))
+                                             .fqdn(dataCenter + "-i" + index)
+                                             .rack("Rack")
+                                             .hostId("")
+                                             .status("UP")
+                                             .state("NORMAL")
+                                             .owns("")
+                                             .build());
+
+        byte[] bytes = serialize(ring);
+        RingInstance deserialized = deserialize(bytes, RingInstance.class);
+        assertEquals(ring, deserialized);
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/RingInstanceTest.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/RingInstanceTest.java
new file mode 100644
index 0000000..3902109
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/RingInstanceTest.java
@@ -0,0 +1,229 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Multimap;
+import com.google.common.collect.Range;
+import org.junit.Test;
+
+import org.apache.cassandra.sidecar.common.data.RingEntry;
+import org.apache.cassandra.spark.bulkwriter.token.CassandraRing;
+import org.apache.cassandra.spark.bulkwriter.token.ConsistencyLevel;
+import org.apache.cassandra.spark.bulkwriter.token.ReplicaAwareFailureHandler;
+import org.apache.cassandra.spark.common.model.CassandraInstance;
+import org.apache.cassandra.spark.data.ReplicationFactor;
+import org.apache.cassandra.spark.data.partitioner.Partitioner;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+public class RingInstanceTest
+{
+    public static final String DATACENTER_1 = "DATACENTER1";
+    public static final String KEYSPACE = "KEYSPACE";
+
+    @Test
+    public void testEquals()
+    {
+        RingInstance instance1 = RingUtils.getInstances(0, ImmutableMap.of("DATACENTER1", 1), 1).get(0);
+        RingInstance instance2 = RingUtils.getInstances(0, ImmutableMap.of("DATACENTER1", 1), 1).get(0);
+        assertEquals(instance1, instance2);
+    }
+
+    @Test
+    public void testHashCode()
+    {
+        RingInstance instance1 = RingUtils.getInstances(0, ImmutableMap.of("DATACENTER1", 1), 1).get(0);
+        RingInstance instance2 = RingUtils.getInstances(0, ImmutableMap.of("DATACENTER1", 1), 1).get(0);
+        assertEquals(instance1.hashCode(), instance2.hashCode());
+    }
+
+    @Test
+    public void testEqualsAndHashcodeIgnoreHost()
+    {
+        RingInstance realInstance = new RingInstance(new RingEntry.Builder()
+                                                     .datacenter("DATACENTER1")
+                                                     .address("127.0.0.1")
+                                                     .port(7000)
+                                                     .rack("Rack")
+                                                     .status("UP")
+                                                     .state("NORMAL")
+                                                     .load("0")
+                                                     .token("0")
+                                                     .fqdn("fqdn")
+                                                     .hostId("")
+                                                     .owns("")
+                                                     .build());
+
+        RingInstance questionInstance = new RingInstance(new RingEntry.Builder()
+                                                         .datacenter("DATACENTER1")
+                                                         .address("127.0.0.1")
+                                                         .port(7000)
+                                                         .rack("Rack")
+                                                         .status("UP")
+                                                         .state("NORMAL")
+                                                         .load("0")
+                                                         .token("0")
+                                                         .fqdn("fqdn")
+                                                         .hostId("")
+                                                         .owns("")
+                                                         .build());
+        assertEquals(realInstance, questionInstance);
+        assertEquals(realInstance.hashCode(), questionInstance.hashCode());
+    }
+
+    @Test
+    public void multiMapWorksWithRingInstances()
+    {
+        RingInstance instance1 = RingUtils.getInstances(0, ImmutableMap.of("DATACENTER1", 1), 1).get(0);
+        RingInstance instance2 = RingUtils.getInstances(0, ImmutableMap.of("DATACENTER1", 1), 1).get(0);
+        byte[] buffer;
+
+        try
+        {
+            ByteArrayOutputStream baos = new ByteArrayOutputStream();
+            ObjectOutputStream out = new ObjectOutputStream(baos);
+            out.writeObject(instance2);
+            out.close();
+            buffer = baos.toByteArray();
+            ByteArrayInputStream bais = new ByteArrayInputStream(buffer);
+            ObjectInputStream in = new ObjectInputStream(bais);
+            instance2 = (RingInstance) in.readObject();
+            in.close();
+        }
+        catch (IOException | ClassNotFoundException exception)
+        {
+            throw new RuntimeException(exception);
+        }
+        Multimap<RingInstance, String> initialErrorMap = ArrayListMultimap.create();
+        initialErrorMap.put(instance1, "Failure 1");
+        Multimap<RingInstance, String> newErrorMap = ArrayListMultimap.create(initialErrorMap);
+        newErrorMap.put(instance2, "Failure2");
+
+        assertEquals(1, newErrorMap.keySet().size());
+    }
+
+    @Test
+    public void testMultipleFailuresSingleInstanceSucceedRF3()
+    {
+        Partitioner partitioner = Partitioner.Murmur3Partitioner;
+        BigInteger[] tokens = getTokens(partitioner, 5);
+        List<CassandraInstance> instances = getInstances(tokens, DATACENTER_1);
+        CassandraInstance instance1 = instances.get(0);
+        CassandraInstance instance2 = instance(tokens[0], instance1.getNodeName(), instance1.getDataCenter(), "?");
+        ReplicaAwareFailureHandler<CassandraInstance> replicationFactor3 =
+                ntsStrategyHandler(partitioner, instances, new String[]{DATACENTER_1}, new int[]{3});
+        // This test proves that for any RF3 keyspace
+        replicationFactor3.addFailure(Range.openClosed(tokens[0], tokens[1]), instance1, "Complete Failure");
+        replicationFactor3.addFailure(Range.openClosed(tokens[0], tokens[0].add(BigInteger.ONE)), instance2, "Failure 1");
+        replicationFactor3.addFailure(Range.openClosed(tokens[0].add(BigInteger.ONE),
+                                                       tokens[0].add(BigInteger.valueOf(2L))), instance2, "Failure 2");
+
+        replicationFactor3.getFailedEntries(ConsistencyLevel.CL.LOCAL_QUORUM, DATACENTER_1);
+        assertFalse(replicationFactor3.hasFailed(ConsistencyLevel.CL.LOCAL_QUORUM, DATACENTER_1));
+    }
+
+    static List<CassandraInstance> getInstances(BigInteger[] tokens, String datacenter)
+    {
+        List<CassandraInstance> instances = new ArrayList<>();
+        for (int token = 0; token < tokens.length; token++)
+        {
+            instances.add(instance(tokens[token], "node-" + token, datacenter, "host"));
+        }
+        return instances;
+    }
+
+    static CassandraInstance instance(BigInteger token, String nodeName, String datacenter, String hostName)
+    {
+        return new RingInstance(new RingEntry.Builder()
+                .datacenter(datacenter)
+                .port(7000)
+                .address(nodeName)
+                .status("UP")
+                .state("NORMAL")
+                .token(token.toString())
+                .fqdn(nodeName)
+                .rack("rack")
+                .owns("")
+                .load("")
+                .hostId("")
+                .build());
+    }
+
+    static BigInteger[] getTokens(Partitioner partitioner, int nodes)
+    {
+        BigInteger[] tokens = new BigInteger[nodes];
+
+        for (int node = 0; node < nodes; node++)
+        {
+            tokens[node] = partitioner == Partitioner.Murmur3Partitioner ? getMurmur3Token(nodes, node)
+                                                                         : getRandomToken(nodes, node);
+        }
+        return tokens;
+    }
+
+    static BigInteger getRandomToken(int nodes, int index)
+    {
+        // ((2^127 / nodes) * i)
+        return ((BigInteger.valueOf(2).pow(127)).divide(BigInteger.valueOf(nodes))).multiply(BigInteger.valueOf(index));
+    }
+
+    static BigInteger getMurmur3Token(int nodes, int index)
+    {
+        // (((2^64 / n) * i) - 2^63)
+        return (((BigInteger.valueOf(2).pow(64)).divide(BigInteger.valueOf(nodes)))
+            .multiply(BigInteger.valueOf(index))).subtract(BigInteger.valueOf(2).pow(63));
+    }
+
+    private static ReplicaAwareFailureHandler<CassandraInstance> ntsStrategyHandler(Partitioner partitioner,
+                                                                                    List<CassandraInstance> instances,
+                                                                                    String[] dataCenters,
+                                                                                    int[] replicationFactors)
+    {
+        ReplicationFactor repFactor = new ReplicationFactor(ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy,
+                                                            ntsOptions(dataCenters, replicationFactors));
+        CassandraRing<CassandraInstance> ring = new CassandraRing<>(partitioner, KEYSPACE, repFactor, instances);
+        return new ReplicaAwareFailureHandler<>(ring);
+    }
+
+    private static Map<String, Integer> ntsOptions(String[] names, int[] values)
+    {
+        assert names.length == values.length : "Invalid replication options - array lengths do not match";
+        Map<String, Integer> options = Maps.newHashMap();
+        for (int name = 0; name < names.length; name++)
+        {
+            options.put(names[name], values[name]);
+        }
+        return options;
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/RingUtils.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/RingUtils.java
new file mode 100644
index 0000000..4916cb8
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/RingUtils.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.cassandra.spark.bulkwriter;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.sidecar.common.data.RingEntry;
+import org.apache.cassandra.spark.bulkwriter.token.CassandraRing;
+import org.apache.cassandra.spark.data.ReplicationFactor;
+import org.apache.cassandra.spark.data.partitioner.Partitioner;
+import org.jetbrains.annotations.NotNull;
+
+public final class RingUtils
+{
+    private RingUtils()
+    {
+        throw new IllegalStateException(getClass() + " is static utility class and shall not be instantiated");
+    }
+
+    @NotNull
+    static CassandraRing<RingInstance> buildRing(int initialToken,
+                                                 String app,
+                                                 String cluster,
+                                                 String dataCenter,
+                                                 String keyspace)
+    {
+        return buildRing(initialToken, app, cluster, dataCenter, keyspace, 3);
+    }
+
+    @NotNull
+    public static CassandraRing<RingInstance> buildRing(int initialToken,
+                                                        String app,
+                                                        String cluster,
+                                                        String dataCenter,
+                                                        String keyspace,
+                                                        int instancesPerDC)
+    {
+        ImmutableMap<String, Integer> rfByDC = ImmutableMap.of(dataCenter, 3);
+        return buildRing(initialToken, app, cluster, rfByDC, keyspace, instancesPerDC);
+    }
+
+    @NotNull
+    static CassandraRing<RingInstance> buildRing(int initialToken,
+                                                 String app,
+                                                 String cluster,
+                                                 ImmutableMap<String, Integer> rfByDC,
+                                                 String keyspace,
+                                                 int instancesPerDC)
+    {
+        List<RingInstance> instances = getInstances(initialToken, rfByDC, instancesPerDC);
+        ReplicationFactor replicationFactor = getReplicationFactor(rfByDC);
+        return new CassandraRing<>(Partitioner.Murmur3Partitioner, keyspace, replicationFactor, instances);
+    }
+
+    @NotNull
+    private static ReplicationFactor getReplicationFactor(Map<String, Integer> rfByDC)
+    {
+        ImmutableMap.Builder<String, String> optionsBuilder = ImmutableMap.<String, String>builder()
+                .put("class", "org.apache.cassandra.locator.NetworkTopologyStrategy");
+        rfByDC.forEach((key, value) -> optionsBuilder.put(key, value.toString()));
+        return new ReplicationFactor(optionsBuilder.build());
+    }
+
+    @NotNull
+    public static List<RingInstance> getInstances(int initialToken, Map<String, Integer> rfByDC, int instancesPerDc)
+    {
+        ArrayList<RingInstance> instances = new ArrayList<>();
+        int dcOffset = 0;
+        for (Map.Entry<String, Integer> rfForDC : rfByDC.entrySet())
+        {
+            String datacenter = rfForDC.getKey();
+            for (int instance = 0; instance < instancesPerDc; instance++)
+            {
+                instances.add(new RingInstance(new RingEntry.Builder()
+                        .address("127.0." + dcOffset + "." + instance)
+                        .datacenter(datacenter)
+                        .load("0")
+                        .token(Integer.toString(initialToken + dcOffset + 100_000 * instance))
+                        .fqdn(datacenter + "-i" + instance)
+                        .rack("Rack")
+                        .hostId("")
+                        .status("UP")
+                        .state("NORMAL")
+                        .owns("")
+                        .build()));
+            }
+            dcOffset++;
+        }
+        return instances;
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/SSTableWriterTest.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/SSTableWriterTest.java
new file mode 100644
index 0000000..8bee484
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/SSTableWriterTest.java
@@ -0,0 +1,103 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+import java.io.IOException;
+import java.math.BigInteger;
+import java.nio.file.DirectoryStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.stream.Collectors;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.cassandra.bridge.CassandraVersion;
+import org.apache.cassandra.bridge.CassandraVersionFeatures;
+import org.apache.cassandra.spark.bulkwriter.token.CassandraRing;
+import org.apache.cassandra.spark.bulkwriter.token.ConsistencyLevel;
+import org.jetbrains.annotations.NotNull;
+
+import static org.junit.Assert.assertEquals;
+
+@RunWith(Parameterized.class)
+public class SSTableWriterTest
+{
+    private static String previousMbeanState;
+
+    @Parameterized.Parameters(name = "{index}: Testing Cassandra Version {0}")
+    public static Iterable<Object[]> data()
+    {
+        return Arrays.stream(CassandraVersion.supportedVersions())
+                     .map(version -> new Object[]{version})
+                     .collect(Collectors.toList());
+    }
+
+    @BeforeClass
+    public static void setProps()
+    {
+        previousMbeanState = System.getProperty("org.apache.cassandra.disable_mbean_registration");
+        System.setProperty("org.apache.cassandra.disable_mbean_registration", "true");
+    }
+
+    @AfterClass
+    public static void restoreProps()
+    {
+        if (previousMbeanState != null)
+        {
+            System.setProperty("org.apache.cassandra.disable_mbean_registration", previousMbeanState);
+        }
+        else
+        {
+            System.clearProperty("org.apache.cassandra.disable_mbean_registration");
+        }
+    }
+
+    @NotNull
+    public CassandraRing<RingInstance> ring = RingUtils.buildRing(0, "app", "cluster", "DC1", "test", 12);  // CHECKSTYLE IGNORE: Public mutable field
+
+    @Rule
+    public TemporaryFolder tmpDir = new TemporaryFolder();
+
+    @Parameterized.Parameter(0)
+    public String version;  // CHECKSTYLE IGNORE: Public mutable field for parameterized testing
+
+    @Test
+    public void canCreateWriterForVersion() throws IOException
+    {
+        MockBulkWriterContext writerContext = new MockBulkWriterContext(ring, version, ConsistencyLevel.CL.LOCAL_QUORUM);
+        SSTableWriter tw = new SSTableWriter(writerContext, tmpDir.getRoot().toPath());
+        tw.addRow(BigInteger.ONE, new Object[]{1, 1, "foo", 1});
+        tw.close(writerContext, 1);
+        try (DirectoryStream<Path> dataFileStream = Files.newDirectoryStream(tw.getOutDir(), "*Data.db"))
+        {
+            dataFileStream.forEach(dataFilePath ->
+                    assertEquals(CassandraVersionFeatures.cassandraVersionFeaturesFromCassandraVersion(version).getMajorVersion(),
+                                 SSTables.cassandraVersionFromTable(dataFilePath).getMajorVersion()));
+        }
+        tw.validateSSTables(writerContext, 1);
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/SSTables.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/SSTables.java
new file mode 100644
index 0000000..7c4d2b7
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/SSTables.java
@@ -0,0 +1,134 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+import java.nio.file.Path;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.StringTokenizer;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Maps;
+
+import org.apache.cassandra.bridge.CassandraVersionFeatures;
+import org.jetbrains.annotations.NotNull;
+
+public final class SSTables
+{
+    private enum ComponentType
+    {
+        VERSION,
+        GENERATION,
+        FORMAT,
+        FILE_COMPONENT
+    }
+
+    private static final String FILE_PATH_SEPARATOR = "-";
+    private static final Map<String, CassandraVersionFeatures> SS_TABLE_VERSIONS = ImmutableMap
+            .<String, CassandraVersionFeatures>builder()
+            .put("ma", new CassandraVersionFeatures(30,  0, null))
+            .put("mb", new CassandraVersionFeatures(30,  7, null))
+            .put("mc", new CassandraVersionFeatures(30,  8, null))
+            .put("md", new CassandraVersionFeatures(30, 18, null))
+            .put("me", new CassandraVersionFeatures(30, 25, null))
+            .put("mf", new CassandraVersionFeatures(30, 18, null))
+            .put("na", new CassandraVersionFeatures(40,  0, null))
+            .put("nb", new CassandraVersionFeatures(40,  0, null))
+            .build();
+
+    private SSTables()
+    {
+        throw new IllegalStateException(getClass() + " is static utility class and shall not be instantiated");
+    }
+
+    private static LinkedHashMap<ComponentType, String> parse(@NotNull Path sstable)
+    {
+        String name = sstable.getFileName().toString();
+        StringTokenizer st = new StringTokenizer(name, FILE_PATH_SEPARATOR);
+        LinkedHashMap<ComponentType, String> reverseOrderComponents = Maps.newLinkedHashMap();
+
+        // Read tokens backwards to determine version
+        Deque<String> tokenStack = new ArrayDeque<>();
+        while (st.hasMoreTokens())
+        {
+            tokenStack.push(st.nextToken());
+        }
+
+        reverseOrderComponents.put(ComponentType.FILE_COMPONENT, tokenStack.pop());
+
+        // In pre 2.2 tables, the next token is the generation but post 2.2 it's the format. If all
+        // the chars in this token are numeric, we can assume a generation, as Cassandra does
+        // (see o.a.c.io.sstable.Descriptor::fromFilename). Otherwise, interpret as the format
+        // identifier and so skip over it and read the generation from the next token.
+        String nextToken = tokenStack.pop();
+        if (!isNumeric(nextToken))
+        {
+            reverseOrderComponents.put(ComponentType.FORMAT, nextToken);
+            nextToken = tokenStack.pop();
+        }
+        reverseOrderComponents.put(ComponentType.GENERATION, nextToken);
+
+        String version = tokenStack.pop();
+        reverseOrderComponents.put(ComponentType.VERSION, version);
+
+        // Finally, reverse the component map to put it in forward order
+        List<ComponentType> keys = new ArrayList<>(reverseOrderComponents.keySet());
+        Collections.reverse(keys);
+        LinkedHashMap<ComponentType, String> components = Maps.newLinkedHashMap();
+        keys.forEach(componentType -> components.put(componentType, reverseOrderComponents.get(componentType)));
+        return components;
+    }
+
+    /**
+     * Replaces guava's `com.google.common.base.CharMatcher.digit().matchesAllOf` to remove the dependency
+     * of guava just for the test
+     *
+     * @param string the input string
+     * @return true if only 0-9 contained in the string, false otherwise
+     */
+    private static boolean isNumeric(String string)
+    {
+        for (int index = 0; index < string.length(); index++)
+        {
+            char character = string.charAt(index);
+            if (character < '0' || '9' < character)
+            {
+                return false;
+            }
+        }
+        return true;
+    }
+
+    public static CassandraVersionFeatures cassandraVersionFromTable(@NotNull Path sstable)
+    {
+        String version = parse(sstable).get(ComponentType.VERSION);
+        if (!SS_TABLE_VERSIONS.containsKey(version))
+        {
+            throw new UnsupportedOperationException("SSTable version: " + version + " is not supported");
+        }
+
+        return SS_TABLE_VERSIONS.get(version);
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/SqlToCqlTypeConverterTest.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/SqlToCqlTypeConverterTest.java
new file mode 100644
index 0000000..9efdb84
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/SqlToCqlTypeConverterTest.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.cassandra.spark.bulkwriter;
+
+import java.util.Arrays;
+import java.util.Collection;
+
+import org.junit.Test;
+import org.junit.experimental.runners.Enclosed;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.cassandra.spark.data.CqlField;
+
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.ASCII;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.BIGINT;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.BLOB;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.BOOLEAN;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.COUNTER;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.DATE;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.DECIMAL;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.DOUBLE;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.FLOAT;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.INET;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.INT;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.SMALLINT;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.TEXT;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.TIME;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.TIMESTAMP;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.TIMEUUID;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.TINYINT;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.UUID;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.VARCHAR;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.VARINT;
+import static org.apache.cassandra.spark.bulkwriter.TableSchemaTestCommon.mockCqlCustom;
+import static org.apache.cassandra.spark.bulkwriter.TableSchemaTestCommon.mockCqlType;
+import static org.apache.cassandra.spark.bulkwriter.TableSchemaTestCommon.mockListCqlType;
+import static org.apache.cassandra.spark.bulkwriter.TableSchemaTestCommon.mockMapCqlType;
+import static org.apache.cassandra.spark.bulkwriter.TableSchemaTestCommon.mockSetCqlType;
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.junit.Assert.assertThat;
+
+@RunWith(Enclosed.class)
+public final class SqlToCqlTypeConverterTest
+{
+    private SqlToCqlTypeConverterTest()
+    {
+        throw new IllegalStateException(getClass() + " is static utility class and shall not be instantiated");
+    }
+
+    @RunWith(Parameterized.class)
+    public static class ConverterTests
+    {
+        public ConverterTests(CqlField.CqlType dt, Class expectedConverter)
+        {
+            this.dt = dt;
+            this.expectedConverter = expectedConverter;
+        }
+
+        @Parameterized.Parameters(name = "{index}: {0} -> {1}")
+        public static Collection<Object[]> dataTypes()
+        {
+            return Arrays.asList(na(mockCqlType(BIGINT), SqlToCqlTypeConverter.LongConverter.class),
+                                 na(mockCqlType(BLOB), SqlToCqlTypeConverter.BytesConverter.class),
+                                 na(mockListCqlType(INT), SqlToCqlTypeConverter.ListConverter.class),
+                                 na(mockCqlType(DECIMAL), SqlToCqlTypeConverter.BigDecimalConverter.class),
+                                 na(mockCqlType(TIMESTAMP), SqlToCqlTypeConverter.TimestampConverter.class),
+                                 na(mockCqlType(TIME), SqlToCqlTypeConverter.TimeConverter.class),
+                                 na(mockCqlType(UUID), SqlToCqlTypeConverter.UUIDConverter.class),
+                                 na(mockCqlType(VARINT), SqlToCqlTypeConverter.BigIntegerConverter.class),
+                                 na(mockCqlType(TIMEUUID), SqlToCqlTypeConverter.TimeUUIDConverter.class),
+                                 na(mockCqlType(INET), SqlToCqlTypeConverter.InetAddressConverter.class),
+                                 na(mockCqlType(DATE), SqlToCqlTypeConverter.DateConverter.class),
+                                 na(mockMapCqlType(INT, INT), SqlToCqlTypeConverter.MapConverter.class),
+                                 na(mockSetCqlType(INET), SqlToCqlTypeConverter.SetConverter.class),
+                                 // Special Cassandra 1.2 Timestamp type should use TimestampConverter
+                                 na(mockCqlCustom("org.apache.cassandra.db.marshal.DateType"), SqlToCqlTypeConverter.TimestampConverter.class));
+        }
+
+        private CqlField.CqlType dt;
+        private Class expectedConverter;
+
+        @Test
+        public void testConverter()
+        {
+            assertConverterType(dt, expectedConverter);
+        }
+
+        private static Object[] na(CqlField.CqlType dataType, Class converter)
+        {
+            return new Object[]{dataType, converter};
+        }
+    }
+
+    @RunWith(Parameterized.class)
+    public static class TypeConverterNoOpTests
+    {
+        @Parameterized.Parameters(name = "{index}: {0}")
+        public static Collection<Object[]> dataTypes()
+        {
+            return Arrays.asList(na(mockCqlType(ASCII)),
+                                 na(mockCqlType(BOOLEAN)),
+                                 na(mockCqlType(COUNTER)),
+                                 na(mockCqlType(DOUBLE)),
+                                 na(mockCqlType(FLOAT)),
+                                 na(mockCqlType(TEXT)),
+                                 na(mockCqlType(VARCHAR)),
+                                 na(mockCqlType(SMALLINT)),
+                                 na(mockCqlType(TINYINT)));
+        }
+
+        @Parameterized.Parameter
+        public CqlField.CqlType dataType;  // CHECKSTYLE IGNORE: Public mutable field for parameterized testing
+
+        private static Object[] na(CqlField.CqlType dataType)
+        {
+            return new Object[]{dataType};
+        }
+
+        @Test
+        public void testNoOpTypes()
+        {
+            assertConverterType(dataType, SqlToCqlTypeConverter.NoOp.class);
+        }
+    }
+
+    public static void assertConverterType(CqlField.CqlType dataType, Class expectedType)
+    {
+        assertThat(SqlToCqlTypeConverter.getConverter(dataType), instanceOf(expectedType));
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/StreamSessionConsistencyTest.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/StreamSessionConsistencyTest.java
new file mode 100644
index 0000000..0c6d5db
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/StreamSessionConsistencyTest.java
@@ -0,0 +1,217 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+import java.io.IOException;
+import java.math.BigInteger;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import com.google.common.collect.BoundType;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Range;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.cassandra.spark.bulkwriter.token.CassandraRing;
+import org.apache.cassandra.spark.bulkwriter.token.ConsistencyLevel;
+import org.apache.cassandra.spark.common.model.CassandraInstance;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.equalTo;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThrows;
+
+@RunWith(Parameterized.class)
+public class StreamSessionConsistencyTest
+{
+    private static final int NUMBER_DCS = 2;
+    private static final int FILES_PER_SSTABLE = 8;
+    private static final int REPLICATION_FACTOR = 3;
+    private static final List<String> EXPECTED_INSTANCES =
+            ImmutableList.of("DC1-i1", "DC1-i2", "DC1-i3", "DC2-i1", "DC2-i2", "DC2-i3");
+    private static final Range<BigInteger> RANGE = Range.range(BigInteger.valueOf(101L), BoundType.CLOSED,
+                                                               BigInteger.valueOf(199L), BoundType.CLOSED);
+    private static final CassandraRing<RingInstance> RING = RingUtils.buildRing(0,
+                                                                                "app",
+                                                                                "cluster",
+                                                                                ImmutableMap.of("DC1", 3, "DC2", 3),
+                                                                                "test",
+                                                                                6);
+
+    @Rule
+    public TemporaryFolder folder = new TemporaryFolder();
+    private MockTableWriter tableWriter;
+    private StreamSession streamSession;
+    private MockBulkWriterContext writerContext;
+    private final MockScheduledExecutorService executor = new MockScheduledExecutorService();
+
+    @Parameterized.Parameter(0)
+    public ConsistencyLevel.CL consistencyLevel;  // CHECKSTYLE IGNORE: Public mutable field for parameterized testing
+
+    @Parameterized.Parameter(1)
+    public List<Integer> failuresPerDc;           // CHECKSTYLE IGNORE: Public mutable field for parameterized testing
+
+    @Parameterized.Parameters(name = "CL: {0}, numFailures: {1}")
+    public static Collection<Object[]> data()
+    {
+        List<ConsistencyLevel.CL> cls = Arrays.stream(ConsistencyLevel.CL.values()).collect(Collectors.toList());
+        List<Integer> failures = IntStream.rangeClosed(0, REPLICATION_FACTOR).boxed().collect(Collectors.toList());
+        List<List<Integer>> failuresPerDc = Lists.cartesianProduct(failures, failures);
+        List<List<Object>> clsToFailures = Lists.cartesianProduct(cls, failuresPerDc);
+        return clsToFailures.stream().map(List::toArray).collect(Collectors.toList());
+    }
+
+    @Before
+    public void setup()
+    {
+        tableWriter = new MockTableWriter(folder.getRoot().toPath());
+        writerContext = new MockBulkWriterContext(RING, "cassandra-4.0.0", consistencyLevel);
+        streamSession = new StreamSession(writerContext, "sessionId", RANGE, executor);
+    }
+
+    @Test
+    public void testConsistencyLevelAndFailureInCommit() throws IOException, ExecutionException, InterruptedException
+    {
+        streamSession = new StreamSession(writerContext, "sessionId", RANGE, executor);
+        AtomicInteger dc1Failures = new AtomicInteger(failuresPerDc.get(0));
+        AtomicInteger dc2Failures = new AtomicInteger(failuresPerDc.get(1));
+        ImmutableMap<String, AtomicInteger> dcFailures = ImmutableMap.of("DC1", dc1Failures, "DC2", dc2Failures);
+        boolean shouldFail = calculateFailure(dc1Failures.get(), dc2Failures.get());
+        // Return successful result for 1st result, failed for the rest
+        writerContext.setCommitResultSupplier((uuids, dc) -> {
+            if (dcFailures.get(dc).getAndDecrement() > 0)
+            {
+                return new DataTransferApi.RemoteCommitResult(false, null, uuids, "");
+            }
+            else
+            {
+                return new DataTransferApi.RemoteCommitResult(true, uuids, null, "");
+            }
+        });
+        SSTableWriter tr = new NonValidatingTestSSTableWriter(tableWriter, folder.getRoot().toPath());
+        Object[] row = {0, 1, "course", 2};
+        tr.addRow(BigInteger.valueOf(102L), row);
+        tr.close(writerContext, 1);
+        streamSession.scheduleStream(tr);
+        if (shouldFail)
+        {
+            RuntimeException exception = assertThrows(RuntimeException.class,
+                                                      () -> streamSession.close());  // Force "execution" of futures
+            assertEquals("Failed to load 1 ranges with " + consistencyLevel
+                       + " for job " + writerContext.job().getId()
+                       + " in phase UploadAndCommit", exception.getMessage());
+        }
+        else
+        {
+            streamSession.close();  // Force "execution" of futures
+        }
+        executor.assertFuturesCalled();
+        assertThat(writerContext.getUploads().values().stream()
+                                                      .mapToInt(Collection::size)
+                                                      .sum(),
+                   equalTo(REPLICATION_FACTOR * NUMBER_DCS * FILES_PER_SSTABLE));
+        List<String> instances = writerContext.getUploads().keySet().stream()
+                                                                    .map(CassandraInstance::getNodeName)
+                                                                    .collect(Collectors.toList());
+        assertThat(instances, containsInAnyOrder(EXPECTED_INSTANCES.toArray()));
+    }
+
+    @Test
+    public void testConsistencyLevelAndFailureInUpload() throws IOException, ExecutionException, InterruptedException
+    {
+        streamSession = new StreamSession(writerContext, "sessionId", RANGE, executor);
+        AtomicInteger dc1Failures = new AtomicInteger(failuresPerDc.get(0));
+        AtomicInteger dc2Failures = new AtomicInteger(failuresPerDc.get(1));
+        int numFailures = dc1Failures.get() + dc2Failures.get();
+        ImmutableMap<String, AtomicInteger> dcFailures = ImmutableMap.of("DC1", dc1Failures, "DC2", dc2Failures);
+        boolean shouldFail = calculateFailure(dc1Failures.get(), dc2Failures.get());
+        writerContext.setUploadSupplier(instance -> dcFailures.get(instance.getDataCenter()).getAndDecrement() <= 0);
+        SSTableWriter tr = new NonValidatingTestSSTableWriter(tableWriter, folder.getRoot().toPath());
+        Object[] row = {0, 1, "course", 2};
+        tr.addRow(BigInteger.valueOf(102L), row);
+        tr.close(writerContext, 1);
+        streamSession.scheduleStream(tr);
+        if (shouldFail)
+        {
+            RuntimeException exception = assertThrows(RuntimeException.class,
+                                                      () -> streamSession.close());  // Force "execution" of futures
+            assertEquals("Failed to load 1 ranges with " + consistencyLevel
+                       + " for job " + writerContext.job().getId()
+                       + " in phase UploadAndCommit", exception.getMessage());
+        }
+        else
+        {
+            streamSession.close();  // Force "execution" of futures
+        }
+        executor.assertFuturesCalled();
+        int totalFilesToUpload = REPLICATION_FACTOR * NUMBER_DCS * FILES_PER_SSTABLE;
+        // Once a file fails to upload, the rest of the components are not attempted
+        int filesSkipped = (numFailures * (FILES_PER_SSTABLE - 1));
+        assertThat(writerContext.getUploads().values().stream()
+                                                      .mapToInt(Collection::size)
+                                                      .sum(),
+                   equalTo(totalFilesToUpload - filesSkipped));
+        List<String> instances = writerContext.getUploads().keySet().stream()
+                                                                    .map(CassandraInstance::getNodeName)
+                                                                    .collect(Collectors.toList());
+        assertThat(instances, containsInAnyOrder(EXPECTED_INSTANCES.toArray()));
+    }
+
+    private boolean calculateFailure(int dc1Failures, int dc2Failures)
+    {
+        // Assumes LOCAL_DC is DC1, given current CL and Failures, should we fail?
+        int localQuorum = REPLICATION_FACTOR / 2 + 1;
+        int localFailuresViolatingQuorum = REPLICATION_FACTOR - localQuorum;
+        int totalInstances = NUMBER_DCS * REPLICATION_FACTOR;
+        int allDcsQuorum = totalInstances / 2 + 1;
+        switch (consistencyLevel)
+        {
+            case ALL:
+                return dc1Failures + dc2Failures > 0;
+            case TWO:
+                return dc1Failures + dc2Failures > (totalInstances - 2);
+            case QUORUM:
+                return dc1Failures + dc2Failures > totalInstances - allDcsQuorum;  // Total instances - quorum
+            case LOCAL_QUORUM:
+                return (dc1Failures > localFailuresViolatingQuorum);
+            case EACH_QUORUM:
+                return (dc1Failures > localFailuresViolatingQuorum) || (dc2Failures > localFailuresViolatingQuorum);
+            case LOCAL_ONE:
+                return dc1Failures > REPLICATION_FACTOR - 1;
+            case ONE:
+                return (dc1Failures + dc2Failures) > totalInstances - 1;
+            default:
+                throw new IllegalArgumentException("CL: " + consistencyLevel + " not supported");
+        }
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/StreamSessionTest.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/StreamSessionTest.java
new file mode 100644
index 0000000..bf79c31
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/StreamSessionTest.java
@@ -0,0 +1,304 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+import java.io.IOException;
+import java.math.BigInteger;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.BoundType;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Range;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import org.apache.cassandra.spark.bulkwriter.token.CassandraRing;
+import org.apache.cassandra.spark.common.model.CassandraInstance;
+import org.jetbrains.annotations.NotNull;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.iterableWithSize;
+import static org.hamcrest.Matchers.startsWith;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertTrue;
+
+public class StreamSessionTest
+{
+    public static final String LOAD_RANGE_ERROR_PREFIX = "Failed to load 1 ranges with LOCAL_QUORUM";
+    @Rule
+    public TemporaryFolder folder = new TemporaryFolder();
+    private static final int FILES_PER_SSTABLE = 8;
+    private static final int REPLICATION_FACTOR = 3;
+    private StreamSession ss;
+    private MockBulkWriterContext writerContext;
+    private List<String> expectedInstances;
+    private CassandraRing<RingInstance> ring;
+    private MockScheduledExecutorService executor;
+    private MockTableWriter tableWriter;
+    private Range<BigInteger> range;
+
+    @Before
+    public void setup()
+    {
+        range = Range.range(BigInteger.valueOf(101L), BoundType.CLOSED, BigInteger.valueOf(199L), BoundType.CLOSED);
+        ring = RingUtils.buildRing(0, "app", "cluster", "DC1", "test", 12);
+        writerContext = getBulkWriterContext();
+        tableWriter = new MockTableWriter(folder.getRoot().toPath());
+        executor = new MockScheduledExecutorService();
+        ss = new StreamSession(writerContext, "sessionId", range, executor);
+        expectedInstances = Lists.newArrayList("DC1-i1", "DC1-i2", "DC1-i3");
+    }
+
+    @Test
+    public void testGetReplicasReturnsCorrectData()
+    {
+        List<RingInstance> replicas = ss.getReplicas();
+        assertNotNull(replicas);
+        List<String> actualInstances = replicas.stream().map(RingInstance::getNodeName).collect(Collectors.toList());
+        assertThat(actualInstances, containsInAnyOrder(expectedInstances.toArray()));
+    }
+
+    @Test
+    public void testScheduleStreamSendsCorrectFilesToCorrectInstances(
+            ) throws IOException, ExecutionException, InterruptedException
+    {
+        SSTableWriter tr = new NonValidatingTestSSTableWriter(tableWriter, folder.getRoot().toPath());
+        Object[] row = {0, 1, "course", 2};
+        tr.addRow(BigInteger.valueOf(102L), row);
+        tr.close(writerContext, 1);
+        ss.scheduleStream(tr);
+        ss.close();  // Force "execution" of futures
+        executor.assertFuturesCalled();
+        assertThat(executor.futures.size(), equalTo(1));  // We only scheduled one SSTable
+        assertThat(writerContext.getUploads().values().stream()
+                                                      .mapToInt(Collection::size)
+                                                      .sum(),
+                   equalTo(REPLICATION_FACTOR * FILES_PER_SSTABLE));
+        List<String> instances = writerContext.getUploads().keySet().stream()
+                                                                    .map(CassandraInstance::getNodeName)
+                                                                    .collect(Collectors.toList());
+        assertThat(instances, containsInAnyOrder(expectedInstances.toArray()));
+    }
+
+    @Test
+    public void testEmptyTokenRangeFails() throws IOException
+    {
+        Exception exception = assertThrows(IllegalStateException.class, () -> ss = new StreamSession(
+                writerContext,
+                "sessionId",
+                Range.range(BigInteger.valueOf(0L), BoundType.CLOSED, BigInteger.valueOf(0L), BoundType.OPEN)));
+        assertThat(exception.getMessage(), startsWith("Partition range [0‥0) is mapping more than one range {}"));
+    }
+
+    @Test
+    public void testMismatchedTokenRangeFails() throws IOException
+    {
+        SSTableWriter tr = new NonValidatingTestSSTableWriter(tableWriter, folder.getRoot().toPath());
+        Object[] row = {0, 1, "course", 2};
+        tr.addRow(BigInteger.valueOf(9999L), row);
+        tr.close(writerContext, 1);
+        IllegalStateException illegalStateException = assertThrows(IllegalStateException.class,
+                                                      () -> ss.scheduleStream(tr));
+        assertEquals(illegalStateException.getMessage(),
+                     "SSTable range [9999‥9999] should be enclosed in the partition range [101‥199]");
+    }
+
+    @Test
+    public void testUploadFailureCallsClean() throws IOException, ExecutionException, InterruptedException
+    {
+        runFailedUpload();
+
+        List<String> actualInstances = writerContext.getCleanedInstances().stream()
+                                                                          .map(CassandraInstance::getNodeName)
+                                                                          .collect(Collectors.toList());
+        assertThat(actualInstances, containsInAnyOrder(expectedInstances.toArray()));
+    }
+
+    @Test
+    public void testUploadFailureSkipsCleanWhenConfigured() throws IOException, ExecutionException, InterruptedException
+    {
+        writerContext.setSkipCleanOnFailures(true);
+        runFailedUpload();
+
+        List<String> actualInstances = writerContext.getCleanedInstances().stream()
+                                                                          .map(CassandraInstance::getNodeName)
+                                                                          .collect(Collectors.toList());
+        assertThat(actualInstances, iterableWithSize(0));
+        List<UploadRequest> uploads = writerContext.getUploads().values().stream()
+                                                                         .flatMap(Collection::stream)
+                                                                         .collect(Collectors.toList());
+        assertTrue(uploads.size() > 0);
+        assertTrue(uploads.stream().noneMatch(u -> u.uploadSucceeded));
+    }
+
+    @Test
+    public void testUploadFailureRefreshesClusterInfo() throws IOException, ExecutionException, InterruptedException
+    {
+        runFailedUpload();
+        assertThat(writerContext.refreshClusterInfoCallCount, equalTo(3));
+    }
+
+    @Test
+    public void testOutDirCreationFailureCleansAllReplicas()
+    {
+        assertThrows(RuntimeException.class, () -> {
+            SSTableWriter tr = new NonValidatingTestSSTableWriter(tableWriter, tableWriter.getOutDir());
+            Object[] row = {0, 1, "course", 2};
+            tr.addRow(BigInteger.valueOf(102L), row);
+            tr.close(writerContext, 1);
+            tableWriter.removeOutDir();
+            ss.scheduleStream(tr);
+            ss.close();
+        });
+
+        List<String> actualInstances = writerContext.getCleanedInstances().stream()
+                                                                          .map(CassandraInstance::getNodeName)
+                                                                          .collect(Collectors.toList());
+        assertThat(actualInstances, containsInAnyOrder(expectedInstances.toArray()));
+    }
+
+    @Test
+    public void unavailableInstancesCreateErrors() throws IOException, ExecutionException, InterruptedException
+    {
+        writerContext.setInstancesAreAvailable(false);
+        ss = new StreamSession(writerContext, "sessionId", range, executor);
+        SSTableWriter tr = new NonValidatingTestSSTableWriter(tableWriter, folder.getRoot().toPath());
+        Object[] row = {0, 1, "course", 2};
+        tr.addRow(BigInteger.valueOf(102L), row);
+        tr.close(writerContext, 1);
+        ss.scheduleStream(tr);
+        assertThrows(LOAD_RANGE_ERROR_PREFIX, RuntimeException.class, () -> ss.close());
+    }
+
+    @Test
+    public void streamWithNoWritersReturnsEmptyStreamResult() throws ExecutionException, InterruptedException
+    {
+        writerContext.setInstancesAreAvailable(false);
+        ss = new StreamSession(writerContext, "sessionId", range, executor);
+        StreamResult result = ss.close();
+        assertThat(result.failures.size(), equalTo(0));
+        assertThat(result.passed.size(), equalTo(0));
+        assertThat(result.sessionID, equalTo("sessionId"));
+        assertThat(result.tokenRange, equalTo(range));
+    }
+
+    @Test
+    public void failedCleanDoesNotThrow() throws IOException, ExecutionException, InterruptedException
+    {
+        writerContext.setCleanShouldThrow(true);
+        runFailedUpload();
+    }
+
+    @Test
+    public void testLocalQuorumSucceedsWhenSingleCommitFails(
+            ) throws IOException, ExecutionException, InterruptedException
+    {
+        ss = new StreamSession(writerContext, "sessionId", range, executor);
+        AtomicBoolean success = new AtomicBoolean(true);
+        writerContext.setCommitResultSupplier((uuids, dc) -> {
+            // Return failed result for 1st result, success for the rest
+            if (success.getAndSet(false))
+            {
+                return new DataTransferApi.RemoteCommitResult(false, uuids, null, "");
+            }
+            else
+            {
+                return new DataTransferApi.RemoteCommitResult(true, null, uuids, "");
+            }
+        });
+        SSTableWriter tr = new NonValidatingTestSSTableWriter(tableWriter, folder.getRoot().toPath());
+        Object[] row = {0, 1, "course", 2};
+        tr.addRow(BigInteger.valueOf(102L), row);
+        tr.close(writerContext, 1);
+        ss.scheduleStream(tr);
+        ss.close();  // Force "execution" of futures
+        executor.assertFuturesCalled();
+        assertThat(writerContext.getUploads().values().stream()
+                                                      .mapToInt(Collection::size)
+                                                      .sum(),
+                   equalTo(REPLICATION_FACTOR * FILES_PER_SSTABLE));
+        List<String> instances = writerContext.getUploads().keySet().stream()
+                                                                    .map(CassandraInstance::getNodeName)
+                                                                    .collect(Collectors.toList());
+        assertThat(instances, containsInAnyOrder(expectedInstances.toArray()));
+    }
+
+    @Test
+    public void testLocalQuorumFailsWhenCommitsFail() throws IOException, ExecutionException, InterruptedException
+    {
+        ss = new StreamSession(writerContext, "sessionId", range, executor);
+        AtomicBoolean success = new AtomicBoolean(true);
+        // Return successful result for 1st result, failed for the rest
+        writerContext.setCommitResultSupplier((uuids, dc) -> {
+            if (success.getAndSet(false))
+            {
+                return new DataTransferApi.RemoteCommitResult(true, null, uuids, "");
+            }
+            else
+            {
+                return new DataTransferApi.RemoteCommitResult(false, uuids, null, "");
+            }
+        });
+        SSTableWriter tr = new NonValidatingTestSSTableWriter(tableWriter, folder.getRoot().toPath());
+        Object[] row = {0, 1, "course", 2};
+        tr.addRow(BigInteger.valueOf(102L), row);
+        tr.close(writerContext, 1);
+        ss.scheduleStream(tr);
+        RuntimeException exception = assertThrows(RuntimeException.class, () -> ss.close());  // Force "execution" of futures
+        assertEquals("Failed to load 1 ranges with LOCAL_QUORUM for job " + writerContext.job().getId()
+                   + " in phase UploadAndCommit", exception.getMessage());
+        executor.assertFuturesCalled();
+        assertThat(writerContext.getUploads().values().stream()
+                                                      .mapToInt(Collection::size)
+                                                      .sum(),
+                   equalTo(REPLICATION_FACTOR * FILES_PER_SSTABLE));
+        List<String> instances = writerContext.getUploads().keySet().stream()
+                                                                    .map(CassandraInstance::getNodeName)
+                                                                    .collect(Collectors.toList());
+        assertThat(instances, containsInAnyOrder(expectedInstances.toArray()));
+    }
+
+    private void runFailedUpload() throws IOException, ExecutionException, InterruptedException
+    {
+        writerContext.setUploadSupplier(instance -> false);
+        SSTableWriter tr = new NonValidatingTestSSTableWriter(tableWriter, folder.getRoot().toPath());
+        Object[] row = {0, 1, "course", 2};
+        tr.addRow(BigInteger.valueOf(102L), row);
+        tr.close(writerContext, 1);
+        ss.scheduleStream(tr);
+        assertThrows(LOAD_RANGE_ERROR_PREFIX, RuntimeException.class, () -> ss.close());
+    }
+
+    @NotNull
+    private MockBulkWriterContext getBulkWriterContext()
+    {
+        return new MockBulkWriterContext(ring);
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/TableSchemaNormalizeTest.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/TableSchemaNormalizeTest.java
new file mode 100644
index 0000000..28dab5f
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/TableSchemaNormalizeTest.java
@@ -0,0 +1,314 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import com.google.common.net.InetAddresses;
+import org.junit.Test;
+
+import org.apache.cassandra.spark.common.schema.ColumnType;
+import org.apache.cassandra.spark.common.schema.ColumnTypes;
+import org.apache.cassandra.spark.common.schema.ListType;
+import org.apache.cassandra.spark.common.schema.MapType;
+import org.apache.cassandra.spark.common.schema.SetType;
+import org.apache.cassandra.spark.data.CqlField;
+import org.apache.spark.sql.types.DataTypes;
+
+import static java.util.AbstractMap.SimpleEntry;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.ASCII;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.BIGINT;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.BLOB;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.BOOLEAN;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.DATE;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.DECIMAL;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.DOUBLE;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.FLOAT;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.INET;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.INT;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.LIST;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.SET;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.SMALLINT;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.TEXT;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.TIME;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.TIMESTAMP;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.TIMEUUID;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.TINYINT;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.UUID;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.VARCHAR;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.VARINT;
+import static org.apache.cassandra.spark.bulkwriter.TableSchemaTestCommon.buildSchema;
+import static org.apache.cassandra.spark.bulkwriter.TableSchemaTestCommon.mockCollectionCqlType;
+import static org.apache.cassandra.spark.bulkwriter.TableSchemaTestCommon.mockCqlType;
+import static org.apache.cassandra.spark.bulkwriter.TableSchemaTestCommon.mockListCqlType;
+import static org.apache.cassandra.spark.bulkwriter.TableSchemaTestCommon.mockMapCqlType;
+import static org.apache.cassandra.spark.bulkwriter.TableSchemaTestCommon.mockSetCqlType;
+import static org.hamcrest.core.Is.is;
+import static org.hamcrest.core.IsEqual.equalTo;
+import static org.junit.Assert.assertThat;
+
+public class TableSchemaNormalizeTest
+{
+    @Test
+    public void testAsciiNormalization()
+    {
+        assertNormalized("ascii", mockCqlType(ASCII), ColumnTypes.STRING, "ascii", "ascii", DataTypes.StringType);
+    }
+
+    @Test
+    public void testBigIntNormalization()
+    {
+        assertNormalized("bigint", mockCqlType(BIGINT), ColumnTypes.INT, 1, 1L, DataTypes.IntegerType);
+    }
+
+    @Test
+    public void testBlobNormalization()
+    {
+        assertNormalized("blob", mockCqlType(BLOB), ColumnTypes.BYTES,
+                         new byte[]{1, 1, 1, 1}, ByteBuffer.wrap(new byte[]{1, 1, 1, 1}), DataTypes.BinaryType);
+    }
+
+    @Test
+    public void testBooleanNormalization()
+    {
+        assertNormalized("boolean", mockCqlType(BOOLEAN), ColumnTypes.BOOLEAN, false, false, DataTypes.BooleanType);
+    }
+
+    @Test
+    public void testDecimalNormalization()
+    {
+        assertNormalized("decimal", mockCqlType(DECIMAL), ColumnTypes.DOUBLE,
+                         BigDecimal.valueOf(1.1), BigDecimal.valueOf(1.1), DataTypes.createDecimalType());
+    }
+
+    @Test
+    public void testDoubleNormalization()
+    {
+        assertNormalized("double", mockCqlType(DOUBLE), ColumnTypes.DOUBLE, 1.1, 1.1, DataTypes.DoubleType);
+    }
+
+    @Test
+    public void testFloatNormalization()
+    {
+        assertNormalized("float", mockCqlType(FLOAT), ColumnTypes.DOUBLE, 1.1f, 1.1f, DataTypes.FloatType);
+    }
+
+    @Test
+    public void testInetNormalization()
+    {
+        assertNormalized("inet", mockCqlType(INET), ColumnTypes.STRING,
+                         "192.168.1.1", InetAddresses.forString("192.168.1.1"), DataTypes.StringType);
+    }
+
+    @Test
+    public void testIntNormalization()
+    {
+        assertNormalized("int", mockCqlType(INT), ColumnTypes.INT, 1, 1, DataTypes.IntegerType);
+    }
+
+    @Test
+    public void testTextNormalization()
+    {
+        assertNormalized("text", mockCqlType(TEXT), ColumnTypes.BYTES, "text", "text", DataTypes.StringType);
+    }
+
+    @Test
+    public void testTimestampNormalization()
+    {
+        assertNormalized("timestamp", mockCqlType(TIMESTAMP), ColumnTypes.LONG,
+                         new Date(1), new Date(1), DataTypes.DateType);
+    }
+
+    @Test
+    public void testUuidNormalization()
+    {
+        assertNormalized("uuid", mockCqlType(UUID), ColumnTypes.UUID,
+                         "382d3b34-22af-4b2a-97a3-ae7dbf9e6abe",
+                         java.util.UUID.fromString("382d3b34-22af-4b2a-97a3-ae7dbf9e6abe"), DataTypes.StringType);
+    }
+
+    @Test
+    public void testVarcharNormalization()
+    {
+        assertNormalized("varchar", mockCqlType(VARCHAR), ColumnTypes.STRING,
+                         "varchar", "varchar", DataTypes.StringType);
+    }
+
+    @Test
+    public void testVarIntNormalization()
+    {
+        assertNormalized("varint", mockCqlType(VARINT), ColumnTypes.INT,
+                         "1", BigInteger.valueOf(1), DataTypes.createDecimalType(38, 0));
+    }
+
+    @Test
+    public void testTimeUuidNormalization()
+    {
+        assertNormalized("timeuuid", mockCqlType(TIMEUUID), ColumnTypes.UUID,
+                         java.util.UUID.fromString("0846b690-ce35-11e7-8871-79b4d1aa8ef2"),
+                         java.util.UUID.fromString("0846b690-ce35-11e7-8871-79b4d1aa8ef2"), DataTypes.StringType);
+    }
+
+    @Test
+    public void testSetNormalization()
+    {
+        Set<String> set = new HashSet<>();
+        set.add("A");
+        set.add("B");
+        set.add("C");
+
+        assertNormalized("set", mockSetCqlType(TEXT), new SetType<>(ColumnTypes.STRING),
+                         set, set, DataTypes.createArrayType(DataTypes.StringType));
+    }
+
+    @Test
+    public void testListNormalization()
+    {
+        assertNormalized("list", mockListCqlType(INT), new ListType<>(ColumnTypes.INT),
+                         Arrays.asList(1, 2, 3), Arrays.asList(1, 2, 3),
+                         DataTypes.createArrayType(DataTypes.IntegerType));
+    }
+
+    @Test
+    public void testMapNormalization()
+    {
+        Map<String, String> map = Stream.of(new SimpleEntry<>("Foo", "Bar"))
+                                        .collect(Collectors.toMap(SimpleEntry::getKey, SimpleEntry::getValue));
+        assertNormalized("map", mockMapCqlType(TEXT, TEXT), new MapType<>(ColumnTypes.STRING, ColumnTypes.STRING),
+                         map, map, DataTypes.createMapType(DataTypes.StringType, DataTypes.StringType));
+    }
+
+    @Test
+    public void testSmallIntNormalization()
+    {
+        assertNormalized("smallint", mockCqlType(SMALLINT), ColumnTypes.INT, (short) 2, (short) 2, DataTypes.ShortType);
+    }
+
+    @Test
+    public void testTinyIntNormalization()
+    {
+        assertNormalized("tiny", mockCqlType(TINYINT), ColumnTypes.INT, (byte) 3, (byte) 3, DataTypes.ByteType);
+    }
+
+    @Test
+    public void testDateNormalization()
+    {
+        assertNormalized("date", mockCqlType(DATE), ColumnTypes.LONG, new Date(2), -2147483648, DataTypes.DateType);
+    }
+
+    @Test
+    public void testTimeNormalizationFromTimestamp()
+    {
+        Timestamp timestamp = new Timestamp(0, 0, 0, 0, 0, 0, 3);
+        assertNormalized("time", mockCqlType(TIME), ColumnTypes.LONG, timestamp, 3L, DataTypes.TimestampType);
+    }
+
+    @Test
+    public void testTimeNormalizationFromLong()
+    {
+        assertNormalized("time", mockCqlType(TIME), ColumnTypes.LONG, 7L, 7L, DataTypes.LongType);
+    }
+
+    @Test
+    public void testByteArrayListNormalization()
+    {
+        assertNormalized("byte_array_list", mockListCqlType(BLOB), new ListType<>(ColumnTypes.BYTES),
+                         Arrays.asList(new byte[]{1}, new byte[]{2}, new byte[]{3}),
+                         Arrays.asList(ByteBuffer.wrap(new byte[]{1}), ByteBuffer.wrap(new byte[]{2}), ByteBuffer.wrap(new byte[]{3})),
+                         DataTypes.createArrayType(DataTypes.BinaryType));
+    }
+
+    @Test
+    public void testByteArrayMapNormalization()
+    {
+        byte[] bytes = {'B', 'a', 'r'};
+
+        Map<String, byte[]> source = Stream.of(new SimpleEntry<>("Foo", bytes))
+                                           .collect(Collectors.toMap(SimpleEntry::getKey, SimpleEntry::getValue));
+        Map<String, ByteBuffer> expected = Stream.of(new SimpleEntry<>("Foo", ByteBuffer.wrap(bytes)))
+                                                 .collect(Collectors.toMap(SimpleEntry::getKey, SimpleEntry::getValue));
+        assertNormalized("mapWithBytes", mockMapCqlType(TEXT, BLOB), new MapType<>(ColumnTypes.STRING, ColumnTypes.STRING),
+                         source, expected, DataTypes.createMapType(DataTypes.StringType, DataTypes.StringType));
+    }
+
+    @Test
+    public void testByteArraySetNormalization()
+    {
+        byte[] bytes = {'B', 'a', 'r'};
+
+        Set<byte[]> source = new HashSet<>(Arrays.asList(new byte[][]{bytes}));
+        Set<ByteBuffer> expected = new HashSet<>(Collections.singletonList(ByteBuffer.wrap(bytes)));
+        assertNormalized("setWithBytes", mockSetCqlType(BLOB), new SetType<>(ColumnTypes.BYTES),
+                         source, expected, DataTypes.createArrayType(DataTypes.BinaryType));
+    }
+
+    @Test
+    public void testNestedNormalization()
+    {
+        byte[] bytes = {'B', 'a', 'r'};
+
+        Map<String, List<Set<byte[]>>> source = new HashMap<>();
+        source.put("Foo1", Arrays.asList(new HashSet<>(Arrays.asList(new byte[][]{bytes})),
+                                         new HashSet<>(Arrays.asList(new byte[][]{bytes}))));
+        source.put("Foo2", Arrays.asList(new HashSet<>(Arrays.asList(new byte[][]{bytes})),
+                                         new HashSet<>(Arrays.asList(new byte[][]{bytes}))));
+
+        Map<String, List<Set<ByteBuffer>>> expected = new HashMap<>();
+        expected.put("Foo1", Arrays.asList(new HashSet<>(Collections.singletonList(ByteBuffer.wrap(bytes))),
+                                           new HashSet<>(Collections.singletonList(ByteBuffer.wrap(bytes)))));
+        expected.put("Foo2", Arrays.asList(new HashSet<>(Collections.singletonList(ByteBuffer.wrap(bytes))),
+                                           new HashSet<>(Collections.singletonList(ByteBuffer.wrap(bytes)))));
+
+        CqlField.CqlMap cqlType = mockMapCqlType(mockCqlType(TEXT),
+                                                 mockCollectionCqlType(LIST, mockCollectionCqlType(SET, mockCqlType(BLOB))));
+        assertNormalized("byte_array_list", cqlType, new MapType<>(ColumnTypes.STRING, new ListType<>(ColumnTypes.BYTES)),
+                         source, expected, DataTypes.createMapType(DataTypes.StringType,
+                                                                   DataTypes.createArrayType(DataTypes.createArrayType(DataTypes.BinaryType))));
+    }
+
+    private void assertNormalized(String field,
+                                  CqlField.CqlType cqlType,
+                                  ColumnType<?> columnType,
+                                  Object sourceVal,
+                                  Object expectedVal,
+                                  org.apache.spark.sql.types.DataType sparkType)
+    {
+        org.apache.spark.sql.types.DataType[] sparkTypes = new org.apache.spark.sql.types.DataType[]{sparkType};
+        String[] fieldNames = {field};
+        ColumnType<?>[] cqlTypes = {columnType};
+        TableSchema schema = buildSchema(fieldNames, sparkTypes, new CqlField.CqlType[]{cqlType}, fieldNames, cqlTypes, fieldNames);
+        Object[] source = new Object[]{sourceVal};
+        Object[] expected = new Object[]{expectedVal};
+        assertThat(schema.normalize(source), is(equalTo(expected)));
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/TableSchemaTest.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/TableSchemaTest.java
new file mode 100644
index 0000000..c045492
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/TableSchemaTest.java
@@ -0,0 +1,199 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+import java.util.Arrays;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.commons.lang3.tuple.Pair;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import org.apache.cassandra.spark.common.schema.ColumnType;
+import org.apache.cassandra.spark.common.schema.ColumnTypes;
+import org.apache.cassandra.spark.data.CqlField;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.StructType;
+
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.DATE;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.INT;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.VARCHAR;
+import static org.apache.cassandra.spark.bulkwriter.TableSchemaTestCommon.mockCqlType;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.core.Is.is;
+import static org.hamcrest.core.IsEqual.equalTo;
+import static org.junit.Assert.assertThrows;
+
+public class TableSchemaTest
+{
+    public TableSchemaTest()
+    {
+        Pair<StructType, ImmutableMap<String, CqlField.CqlType>> validPair = TableSchemaTestCommon.buildMatchedDataframeAndCqlColumns(
+                new String[]{"id", "date", "course", "marks"},
+                new org.apache.spark.sql.types.DataType[]{DataTypes.IntegerType, DataTypes.TimestampType, DataTypes.StringType, DataTypes.IntegerType},
+                new CqlField.CqlType[]{mockCqlType(INT), mockCqlType(DATE), mockCqlType(VARCHAR), mockCqlType(INT)});
+        validDataFrameSchema = validPair.getKey();
+        validCqlColumns = validPair.getValue();
+    }
+
+    @Rule
+    public ExpectedException exception = ExpectedException.none();
+
+    private StructType validDataFrameSchema;
+
+    private ImmutableMap<String, CqlField.CqlType> validCqlColumns;
+
+    private final String[] partitionKeyColumns = {"id"};
+    private final String[] primaryKeyColumnNames = {"id", "date"};
+    private final ColumnType<?>[] partitionKeyColumnTypes = {ColumnTypes.INT};
+    private final String cassandraVersion = "cassandra-4.0.2";
+
+    @Test
+    public void testInsertStatement()
+    {
+        TableSchema schema = getValidSchemaBuilder()
+                .build();
+        assertThat(schema.modificationStatement,
+                   is(equalTo("INSERT INTO test.test (id,date,course,marks) VALUES (?,?,?,?);")));
+    }
+
+    @Test
+    public void testDeleteStatement()
+    {
+        Pair<StructType, ImmutableMap<String, CqlField.CqlType>> validPair = TableSchemaTestCommon.buildMatchedDataframeAndCqlColumns(
+                new String[]{"id"},
+                new org.apache.spark.sql.types.DataType[]{DataTypes.IntegerType},
+                new CqlField.CqlType[]{mockCqlType(INT)});
+        validDataFrameSchema = validPair.getKey();
+        validCqlColumns = validPair.getValue();
+        TableSchema schema = getValidSchemaBuilder()
+                .withWriteMode(WriteMode.DELETE_PARTITION)
+                .build();
+        assertThat(schema.modificationStatement, is(equalTo("DELETE FROM test.test where id=?;")));
+    }
+
+    @Test
+    public void testDeleteWithNonPartitionKeyFieldsInDfFails()
+    {
+        IllegalArgumentException iex = assertThrows(IllegalArgumentException.class, () -> getValidSchemaBuilder()
+                .withWriteMode(WriteMode.DELETE_PARTITION)
+                .build());
+        assertThat(iex.getMessage(),
+                   is(equalTo("Only partition key columns (id) are supported in the input Dataframe when "
+                            + "WRITE_MODE=DELETE_PARTITION but (id,date,course,marks) columns were provided")));
+    }
+
+    @Test
+    public void testPartitionKeyColumnNames()
+    {
+        TableSchema schema = getValidSchemaBuilder()
+                .build();
+        assertThat(schema.partitionKeyColumns, is(equalTo(Arrays.asList("id"))));
+    }
+
+    @Test
+    public void testPartitionKeyColumnTypes()
+    {
+        TableSchema schema = getValidSchemaBuilder()
+                .build();
+        assertThat(schema.partitionKeyColumnTypes, is(equalTo(Arrays.asList(ColumnTypes.INT))));
+    }
+
+    @Test
+    public void normalizeConvertsValidTable()
+    {
+        TableSchema schema = getValidSchemaBuilder()
+                .build();
+
+        assertThat(schema.normalize(new Object[]{1, 1L, "foo", 2}), is(equalTo(new Object[]{1, -2147483648, "foo", 2})));
+    }
+
+    @Test
+    public void testExtraFieldsInDataFrameFails()
+    {
+        StructType extraFieldsDataFrameSchema = new StructType()
+                .add("id", DataTypes.IntegerType)
+                .add("date", DataTypes.TimestampType)
+                .add("extra_field", DataTypes.StringType)
+                .add("course", DataTypes.StringType)
+                .add("marks", DataTypes.IntegerType);
+
+        exception.expect(IllegalArgumentException.class);
+        exception.expectMessage("Unknown fields");
+        getValidSchemaBuilder()
+                .withDataFrameSchema(extraFieldsDataFrameSchema)
+                .build();
+    }
+
+    @Test
+    public void testGetKeyColumnsFindsCorrectValues()
+    {
+        StructType outOfOrderDataFrameSchema = new StructType()
+                .add("date", DataTypes.TimestampType)
+                .add("id", DataTypes.IntegerType)
+                .add("course", DataTypes.StringType)
+                .add("marks", DataTypes.IntegerType);
+
+        TableSchema schema = getValidSchemaBuilder()
+                .withDataFrameSchema(outOfOrderDataFrameSchema)
+                .build();
+        assertThat(schema.getKeyColumns(new Object[]{"date_val", "id_val", "course_val", "marks_val"}),
+                   is(equalTo(new Object[]{"id_val", "date_val"})));
+    }
+
+    @Test
+    public void testGetKeyColumnsFailsWhenNullKeyValues()
+    {
+        TableSchema schema = getValidSchemaBuilder()
+                .build();
+        NullPointerException npe = assertThrows(NullPointerException.class,
+                                                () -> schema.getKeyColumns(new Object[]{"foo", null, "baz", "boo"}));
+        assertThat(npe.getMessage(),
+                   is(equalTo("Found a null primary or composite key column in source data. All key columns must be non-null.")));
+    }
+
+    @Test
+    public void testMissingPrimaryKeyFieldFails()
+    {
+        StructType missingFieldsDataFrame = new StructType()
+                .add("id", DataTypes.IntegerType)
+                .add("course", DataTypes.StringType)
+                .add("marks", DataTypes.IntegerType);
+
+        IllegalArgumentException iex = assertThrows(IllegalArgumentException.class, () -> getValidSchemaBuilder()
+                .withWriteMode(WriteMode.INSERT)
+                .withDataFrameSchema(missingFieldsDataFrame)
+                .build());
+        assertThat(iex.getMessage(), is(equalTo("Missing some required key components in DataFrame => date")));
+    }
+
+    private TableSchemaTestCommon.MockTableSchemaBuilder getValidSchemaBuilder()
+    {
+        return new TableSchemaTestCommon.MockTableSchemaBuilder()
+                .withCqlColumns(validCqlColumns)
+                .withPartitionKeyColumns(partitionKeyColumns)
+                .withPrimaryKeyColumnNames(primaryKeyColumnNames)
+                .withCassandraVersion(cassandraVersion)
+                .withPartitionKeyColumnTypes(partitionKeyColumnTypes)
+                .withWriteMode(WriteMode.INSERT)
+                .withDataFrameSchema(validDataFrameSchema);
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/TableSchemaTestCommon.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/TableSchemaTestCommon.java
new file mode 100644
index 0000000..f393f35
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/TableSchemaTestCommon.java
@@ -0,0 +1,348 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.stream.Collectors;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import org.apache.commons.lang3.tuple.Pair;
+
+import org.apache.cassandra.spark.common.schema.ColumnType;
+import org.apache.cassandra.spark.data.CqlField;
+import org.apache.spark.sql.types.StructType;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.CUSTOM;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.LIST;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.MAP;
+import static org.apache.cassandra.spark.bulkwriter.SqlToCqlTypeConverter.SET;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public final class TableSchemaTestCommon
+{
+    private TableSchemaTestCommon()
+    {
+        throw new IllegalStateException(getClass() + " is static utility class and shall not be instantiated");
+    }
+
+    public static Pair<StructType, ImmutableMap<String, CqlField.CqlType>> buildMatchedDataframeAndCqlColumns(
+            String[] fieldNames,
+            org.apache.spark.sql.types.DataType[] sparkTypes,
+            CqlField.CqlType[] cqlTypes)
+    {
+        StructType dataFrameSchema = new StructType();
+        ImmutableMap.Builder<String, CqlField.CqlType> cqlColumnsBuilder = ImmutableMap.builder();
+        for (int field = 0; field < fieldNames.length; field++)
+        {
+            dataFrameSchema = dataFrameSchema.add(fieldNames[field], sparkTypes[field]);
+            cqlColumnsBuilder.put(fieldNames[field], cqlTypes[field]);
+        }
+
+        ImmutableMap<String, CqlField.CqlType> cqlColumns = cqlColumnsBuilder.build();
+        return Pair.of(dataFrameSchema, cqlColumns);
+    }
+
+    @NotNull
+    public static CqlField.CqlType mockCqlType(String cqlName)
+    {
+        CqlField.CqlType mock = mock(CqlField.CqlType.class);
+        when(mock.name()).thenReturn(cqlName);
+        return mock;
+    }
+
+    @NotNull
+    public static CqlField.CqlCustom mockCqlCustom(String customTypeClassName)
+    {
+        CqlField.CqlCustom mock = mock(CqlField.CqlCustom.class);
+        when(mock.name()).thenReturn(CUSTOM);
+        when(mock.customTypeClassName()).thenReturn(customTypeClassName);
+        return mock;
+    }
+
+    @NotNull
+    public static CqlField.CqlCollection mockSetCqlType(String collectionCqlType)
+    {
+        return mockCollectionCqlType(SET, mockCqlType(collectionCqlType));
+    }
+
+    @NotNull
+    public static CqlField.CqlCollection mockListCqlType(String collectionCqlType)
+    {
+        return mockCollectionCqlType(LIST, mockCqlType(collectionCqlType));
+    }
+
+    @NotNull
+    public static CqlField.CqlCollection mockCollectionCqlType(String cqlName, CqlField.CqlType collectionType)
+    {
+        CqlField.CqlCollection mock = mock(CqlField.CqlCollection.class);
+        when(mock.name()).thenReturn(cqlName);
+        when(mock.type()).thenReturn(collectionType);
+        return mock;
+    }
+
+    @NotNull
+    public static CqlField.CqlType mockMapCqlType(String keyCqlName, String valueCqlName)
+    {
+        return mockMapCqlType(mockCqlType(keyCqlName), mockCqlType(valueCqlName));
+    }
+
+    @NotNull
+    public static CqlField.CqlMap mockMapCqlType(CqlField.CqlType keyType, CqlField.CqlType valueType)
+    {
+        CqlField.CqlMap mock = mock(CqlField.CqlMap.class);
+        when(mock.name()).thenReturn(MAP);
+        when(mock.keyType()).thenReturn(keyType);
+        when(mock.valueType()).thenReturn(valueType);
+        return mock;
+    }
+
+    public static TableSchema buildSchema(String[] fieldNames,
+                                          org.apache.spark.sql.types.DataType[] sparkTypes,
+                                          CqlField.CqlType[] driverTypes,
+                                          String[] partitionKeyColumns,
+                                          ColumnType<?>[] partitionKeyColumnTypes,
+                                          String[] primaryKeyColumnNames)
+    {
+        Pair<StructType, ImmutableMap<String, CqlField.CqlType>> pair = buildMatchedDataframeAndCqlColumns(fieldNames, sparkTypes, driverTypes);
+        ImmutableMap<String, CqlField.CqlType> cqlColumns = pair.getValue();
+        StructType dataFrameSchema = pair.getKey();
+        return
+            new MockTableSchemaBuilder()
+                .withCqlColumns(cqlColumns)
+                .withPartitionKeyColumns(partitionKeyColumns)
+                .withPrimaryKeyColumnNames(primaryKeyColumnNames)
+                .withCassandraVersion("3.0.24.8")
+                .withPartitionKeyColumnTypes(partitionKeyColumnTypes)
+                .withWriteMode(WriteMode.INSERT)
+                .withDataFrameSchema(dataFrameSchema)
+                .build();
+    }
+
+    public static class MockTableSchemaBuilder
+    {
+        private ImmutableMap<String, CqlField.CqlType> cqlColumns;
+        private String[] partitionKeyColumns;
+        private String[] primaryKeyColumnNames;
+        private String cassandraVersion;
+        private ColumnType[] partitionKeyColumnTypes;
+        private StructType dataFrameSchema;
+        private WriteMode writeMode = null;
+
+        public MockTableSchemaBuilder withCqlColumns(@NotNull Map<String, CqlField.CqlType> cqlColumns)
+        {
+            Preconditions.checkNotNull(cqlColumns, "cqlColumns cannot be null");
+            Preconditions.checkArgument(cqlColumns.size() > 0, "cqlColumns cannot be empty");
+            this.cqlColumns = ImmutableMap.copyOf(cqlColumns);
+            return this;
+        }
+
+        public MockTableSchemaBuilder withPartitionKeyColumns(@NotNull String... partitionKeyColumns)
+        {
+            Preconditions.checkNotNull(partitionKeyColumns, "partitionKeyColumns cannot be null");
+            Preconditions.checkArgument(partitionKeyColumns.length > 0, "partitionKeyColumns cannot be empty");
+            this.partitionKeyColumns = partitionKeyColumns;
+            return this;
+        }
+
+        public MockTableSchemaBuilder withPrimaryKeyColumnNames(@NotNull String... primaryKeyColumnNames)
+        {
+            Preconditions.checkNotNull(primaryKeyColumnNames, "primaryKeyColumnNames cannot be null");
+            Preconditions.checkArgument(primaryKeyColumnNames.length > 0, "primaryKeyColumnNames cannot be empty");
+            this.primaryKeyColumnNames = primaryKeyColumnNames;
+            return this;
+        }
+
+        public MockTableSchemaBuilder withCassandraVersion(@NotNull String cassandraVersion)
+        {
+            Preconditions.checkNotNull(cassandraVersion, "cassandraVersion cannot be null");
+            Preconditions.checkArgument(cassandraVersion.length() > 0, "cassandraVersion cannot be an empty string");
+            this.cassandraVersion = cassandraVersion;
+            return this;
+        }
+
+        public MockTableSchemaBuilder withPartitionKeyColumnTypes(@NotNull ColumnType<?>... partitionKeyColumnTypes)
+        {
+            Preconditions.checkNotNull(partitionKeyColumnTypes, "partitionKeyColumnTypes cannot be null");
+            Preconditions.checkArgument(partitionKeyColumnTypes.length > 0, "partitionKeyColumnTypes cannot be empty");
+            this.partitionKeyColumnTypes = Arrays.copyOf(partitionKeyColumnTypes, partitionKeyColumnTypes.length);
+            return this;
+        }
+
+        public MockTableSchemaBuilder withWriteMode(@NotNull WriteMode writeMode)
+        {
+            Preconditions.checkNotNull(writeMode, "writeMode cannot be null");
+            this.writeMode = writeMode;
+            return this;
+        }
+
+        public MockTableSchemaBuilder withDataFrameSchema(StructType dataFrameSchema)
+        {
+            Preconditions.checkNotNull(dataFrameSchema, "dataFrameSchema cannot be null");
+            Preconditions.checkArgument(dataFrameSchema.nonEmpty(), "dataFrameSchema cannot be empty");
+            this.dataFrameSchema = dataFrameSchema;
+            return this;
+        }
+
+        public TableSchema build()
+        {
+            Objects.requireNonNull(cqlColumns,
+                                   "cqlColumns cannot be null. Please provide a list of columns by calling #withCqlColumns");
+            Objects.requireNonNull(partitionKeyColumns,
+                                   "partitionKeyColumns cannot be null. Please provide a list of columns by calling #withPartitionKeyColumns");
+            Objects.requireNonNull(primaryKeyColumnNames,
+                                   "primaryKeyColumnNames cannot be null. Please provide a list of columns by calling #withPrimaryKeyColumnNames");
+            Objects.requireNonNull(cassandraVersion,
+                                   "cassandraVersion cannot be null. Please provide a list of columns by calling #withCassandraVersion");
+            Objects.requireNonNull(partitionKeyColumnTypes,
+                                   "partitionKeyColumnTypes cannot be null. Please provide a list of columns by calling #withPartitionKeyColumnTypes");
+            Objects.requireNonNull(writeMode,
+                                   "writeMode cannot be null. Please provide the write mode by calling #withWriteMode");
+            Objects.requireNonNull(dataFrameSchema,
+                                   "dataFrameSchema cannot be null. Please provide the write mode by calling #withDataFrameSchema");
+            MockTableInfoProvider tableInfoProvider = new MockTableInfoProvider(cqlColumns,
+                                                                                partitionKeyColumns,
+                                                                                partitionKeyColumnTypes,
+                                                                                primaryKeyColumnNames,
+                                                                                cassandraVersion);
+            return new TableSchema(dataFrameSchema, tableInfoProvider, writeMode);
+        }
+    }
+
+    public static class MockTableInfoProvider implements TableInfoProvider
+    {
+        private final ImmutableMap<String, CqlField.CqlType> cqlColumns;
+        private final String[] partitionKeyColumns;
+        private final ColumnType[] partitionKeyColumnTypes;
+        private final String[] primaryKeyColumnNames;
+        Map<String, CqlField.CqlType> columns;
+        private final String cassandraVersion;
+
+        public MockTableInfoProvider(ImmutableMap<String, CqlField.CqlType> cqlColumns,
+                                     String[] partitionKeyColumns,
+                                     ColumnType[] partitionKeyColumnTypes,
+                                     String[] primaryKeyColumnNames,
+                                     String cassandraVersion)
+        {
+            this.cqlColumns = cqlColumns;
+            this.partitionKeyColumns = partitionKeyColumns;
+            this.partitionKeyColumnTypes = partitionKeyColumnTypes;
+            this.primaryKeyColumnNames = primaryKeyColumnNames;
+            columns = cqlColumns;
+            this.cassandraVersion = cassandraVersion.replaceAll("(\\w+-)*cassandra-", "");
+        }
+
+        @Override
+        public CqlField.CqlType getColumnType(String columnName)
+        {
+            return columns.get(columnName);
+        }
+
+        @Override
+        public List<ColumnType<?>> getPartitionKeyTypes()
+        {
+            return Lists.newArrayList(partitionKeyColumnTypes);
+        }
+
+        @Override
+        public boolean columnExists(String columnName)
+        {
+            return columns.containsKey(columnName);
+        }
+
+        @Override
+        public List<String> getPartitionKeyColumnNames()
+        {
+            return Arrays.asList(partitionKeyColumns);
+        }
+
+        @Override
+        public String getCreateStatement()
+        {
+            String keyDef = getKeyDef();
+            String createStatement = "CREATE TABLE test.test (" + cqlColumns.entrySet()
+                                            .stream()
+                                            .map(column -> column.getKey() + " " + column.getValue().name())
+                                            .collect(Collectors.joining(",\n")) + ", " + keyDef + ") "
+                                   + "WITH COMPRESSION = {'class': '" + getCompression() + "'};";
+            System.out.println("Create Table:" + createStatement);
+            return createStatement;
+        }
+
+        private String getCompression()
+        {
+            switch (cassandraVersion.charAt(0))
+            {
+                case '4':
+                    return "ZstdCompressor";
+                case '3':
+                    return "LZ4Compressor";
+                default:
+                    return "LZ4Compressor";
+            }
+        }
+
+        private String getKeyDef()
+        {
+            List<String> partitionColumns = Lists.newArrayList(partitionKeyColumns);
+            List<String> primaryColumns = Lists.newArrayList(primaryKeyColumnNames);
+            primaryColumns.removeAll(partitionColumns);
+            String partitionKey = "(" + String.join(",", partitionKeyColumns) + ")";
+            String clusteringKey = String.join(",", primaryColumns);
+            return "PRIMARY KEY (" + partitionKey + clusteringKey + ")";
+        }
+
+        @Override
+        public List<String> getPrimaryKeyColumnNames()
+        {
+            return Arrays.asList(primaryKeyColumnNames);
+        }
+
+        @Override
+        public String getName()
+        {
+            return "test";
+        }
+
+        @Override
+        public String getKeyspaceName()
+        {
+            return "test";
+        }
+
+        @Override
+        public boolean hasSecondaryIndex()
+        {
+            return false;
+        }
+
+        @Override
+        public List<String> getColumnNames()
+        {
+            return cqlColumns.keySet().asList();
+        }
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/TokenPartitionerTest.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/TokenPartitionerTest.java
new file mode 100644
index 0000000..df4a67f
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/TokenPartitionerTest.java
@@ -0,0 +1,155 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+
+import com.google.common.collect.ImmutableMap;
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.cassandra.spark.bulkwriter.token.CassandraRing;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.number.OrderingComparison.greaterThanOrEqualTo;
+import static org.junit.Assert.assertEquals;
+
+public class TokenPartitionerTest
+{
+    private TokenPartitioner partitioner;
+
+    @Before
+    public void createConfig()
+    {
+    }
+
+    @Test
+    public void testOneSplit()
+    {
+        CassandraRing<RingInstance> ring = RingUtils.buildRing(0, "app", "cluster", "DC1", "test");
+        partitioner = new TokenPartitioner(ring, 1, 2, 1, false);
+        assertEquals(4, partitioner.numPartitions());
+        assertEquals(0, getPartitionForToken(new BigInteger("-9223372036854775808")));
+        assertEquals(0, getPartitionForToken(0));
+        assertEquals(1, getPartitionForToken(1));
+        assertEquals(2, getPartitionForToken(100_001));
+        assertEquals(3, getPartitionForToken(200_001));
+        assertEquals(3, getPartitionForToken(new BigInteger("9223372036854775807")));
+    }
+
+    @Test
+    public void testTwoSplits()
+    {
+        CassandraRing<RingInstance> ring = RingUtils.buildRing(0, "app", "cluster", "DC1", "test");
+        partitioner = new TokenPartitioner(ring, 2, 2, 1, false);
+        assertEquals(10, partitioner.numPartitions());
+        assertEquals(0, getPartitionForToken(new BigInteger("-4611686018427387905")));
+        assertEquals(1, getPartitionForToken(new BigInteger("-4611686018427387904")));
+        assertEquals(1, getPartitionForToken(-1));
+        assertEquals(2, getPartitionForToken(0));  // Single token range
+        assertEquals(3, getPartitionForToken(1));
+        assertEquals(3, getPartitionForToken(50));
+        assertEquals(4, getPartitionForToken(51000));
+        assertEquals(4, getPartitionForToken(51100));
+        assertEquals(5, getPartitionForToken(100001));
+        assertEquals(5, getPartitionForToken(100150));
+        assertEquals(5, getPartitionForToken(150000));
+        assertEquals(6, getPartitionForToken(150001));
+        assertEquals(6, getPartitionForToken(200000));
+        assertEquals(7, getPartitionForToken(200001));
+        assertEquals(7, getPartitionForToken(new BigInteger("4611686018427388003")));
+        assertEquals(7, getPartitionForToken(new BigInteger("4611686018427487903")));
+        assertEquals(8, getPartitionForToken(new BigInteger("4611686018427487904")));
+        assertEquals(9, getPartitionForToken(new BigInteger("9223372036854775807")));  // Single token range
+    }
+
+    // It is possible for a keyspace to replicate to fewer than all datacenters. In these cases, the
+    // check for partitions > instances is incorrect, because it was using the total number of instances
+    // in the cluster (ring.instances), not the number of instances included in the RF of the keyspace.
+    // Instead, we check ring.getTokenRanges().keySet().size(), which returns the list of unique instances
+    // actually participating in the replication of data for this keyspace.
+    // Without the fix, this test would throw during validation.
+    @Test
+    public void testReplicationFactorInOneDCOnly()
+    {
+        CassandraRing<RingInstance> ring = RingUtils.buildRing(0, "app", "cluster", ImmutableMap.of("DC1", 3, "DC2", 0), "test", 3);
+        partitioner = new TokenPartitioner(ring, 1, 2, 1, false);
+        assertEquals(4, partitioner.numPartitions());
+        assertEquals(0, getPartitionForToken(new BigInteger("-9223372036854775808")));
+        assertEquals(0, getPartitionForToken(0));
+        assertEquals(1, getPartitionForToken(100000));
+        assertEquals(2, getPartitionForToken(100001));
+        assertEquals(3, getPartitionForToken(200001));
+        assertEquals(3, getPartitionForToken(new BigInteger("9223372036854775807")));
+    }
+
+    @Test
+    public void testSplitCalculationsUsingCores()
+    {
+        CassandraRing<RingInstance> ring = RingUtils.buildRing(0, "app", "cluster", "DC1", "test");
+        // When passed "-1" for numberSplits, the token partitioner should calculate it on its own based on
+        // the number of cores. This ring has 4 ranges when no splits are used, therefore we expect the number
+        // of splits to be 25 for 100 cores and a default parallelism of 50 (as we take the max of the two).
+        // This results in slightly over 100 partitions, which is what we're looking for.
+        partitioner = new TokenPartitioner(ring, -1, 50, 100, false);
+        assertEquals(25, partitioner.numSplits());
+        assertThat(partitioner.numPartitions(), greaterThanOrEqualTo(100));
+    }
+
+    @Test
+    public void testSplitCalculationsUsingDefaultParallelism()
+    {
+        CassandraRing<RingInstance> ring = RingUtils.buildRing(0, "app", "cluster", "DC1", "test");
+        // When passed "-1" for numberSplits, the token partitioner should calculate it on its own based on
+        // the number of cores. This ring has 4 ranges when no splits are used, therefore we expect the number
+        // of splits to be 50 for 100 cores and a default parallelism of 200 (as we take the max of the two).
+        // This results in slightly over 200 partitions, which is what we're looking for.
+        partitioner = new TokenPartitioner(ring, -1, 200, 100, false);
+        assertEquals(50, partitioner.numSplits());
+        assertThat(partitioner.numPartitions(), greaterThanOrEqualTo(200));
+    }
+
+    @Test
+    public void testSplitCalculationWithMultipleDcs()
+    {
+        ImmutableMap<String, Integer> dcMap = ImmutableMap.<String, Integer>builder()
+                                                                .put("DC1", 3)
+                                                                .put("DC2", 3)
+                                                                .put("DC3", 3)
+                                                                .put("DC4", 3)
+                                                                .build();
+        CassandraRing<RingInstance> ring = RingUtils.buildRing(0, "app", "cluster", dcMap, "test", 20);
+        assertEquals(80, ring.getInstances().size());
+        partitioner = new TokenPartitioner(ring, -1, 1, 750, false);
+        assertEquals(10, partitioner.numSplits());
+        assertThat(partitioner.numPartitions(), greaterThanOrEqualTo(200));
+    }
+
+    private int getPartitionForToken(int token)
+    {
+        return getPartitionForToken(BigInteger.valueOf(token));
+    }
+
+    private int getPartitionForToken(BigInteger token)
+    {
+        return partitioner.getPartition(new DecoratedKey(token, ByteBuffer.allocate(0)));
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/UploadRequest.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/UploadRequest.java
new file mode 100644
index 0000000..637ad2d
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/UploadRequest.java
@@ -0,0 +1,50 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+import java.nio.file.Path;
+
+import org.apache.cassandra.spark.common.MD5Hash;
+import org.apache.cassandra.spark.common.model.CassandraInstance;
+
+public class UploadRequest
+{
+    public final Path path;
+    public final int ssTableIdx;
+    public final CassandraInstance instance;
+    public final String sesssionId;
+    public final MD5Hash fileHash;
+    public final boolean uploadSucceeded;
+
+    public UploadRequest(Path path,
+                         int ssTableIdx,
+                         CassandraInstance instance,
+                         String sesssionId,
+                         MD5Hash fileHash,
+                         boolean uploadSucceeded)
+    {
+        this.path = path;
+        this.ssTableIdx = ssTableIdx;
+        this.instance = instance;
+        this.sesssionId = sesssionId;
+        this.fileHash = fileHash;
+        this.uploadSucceeded = uploadSucceeded;
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/benchmarks/ByteOperationsBenchmark.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/benchmarks/ByteOperationsBenchmark.java
new file mode 100644
index 0000000..c565104
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/bulkwriter/benchmarks/ByteOperationsBenchmark.java
@@ -0,0 +1,78 @@
+/*
+ * 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.cassandra.spark.bulkwriter.benchmarks;
+
+import org.apache.cassandra.spark.bulkwriter.util.FastByteOperations;
+import org.apache.cassandra.spark.utils.RandomUtils;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.Warmup;
+
+import java.nio.ByteBuffer;
+
+@Warmup(iterations = 2)
+@Measurement(iterations = 3)
+@BenchmarkMode(Mode.Throughput)
+@State(Scope.Thread)
+@SuppressWarnings("unused")
+public class ByteOperationsBenchmark
+{
+    @Param({"1", "10", "100", "1000"})
+    int numBytes;
+    ByteBuffer bytes1;
+    ByteBuffer bytes2;
+    FastByteOperations.PureJavaOperations pureJavaOperations = new FastByteOperations.PureJavaOperations();
+    FastByteOperations.UnsafeOperations unsafeOperations = new FastByteOperations.UnsafeOperations();
+
+    @Setup(Level.Trial)
+    public void setup()
+    {
+        byte[] bytes = RandomUtils.randomBytes(numBytes);
+        bytes1 = ByteBuffer.wrap(bytes.clone());
+        bytes2 = ByteBuffer.wrap(bytes.clone());
+    }
+
+    @Benchmark
+    public int unsafeImplementation()
+    {
+        bytes1.position(0);
+        bytes2.position(0);
+        int result = unsafeOperations.compare(bytes1, bytes2);
+        assert 0 == result : "Failed to compare";
+        return result;
+    }
+
+    @Benchmark
+    public int javaImplementation()
+    {
+        bytes1.position(0);
+        bytes2.position(0);
+        int result = pureJavaOperations.compare(bytes1, bytes2);
+        assert result == 0 : "Failed to compare";
+        return result;
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/cdc/watermarker/WatermarkerTests.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/cdc/watermarker/WatermarkerTests.java
new file mode 100644
index 0000000..dd35d93
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/cdc/watermarker/WatermarkerTests.java
@@ -0,0 +1,205 @@
+/*
+ * 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.cassandra.spark.cdc.watermarker;
+
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.spark.TestDataLayer;
+import org.apache.cassandra.spark.cdc.CommitLog;
+import org.apache.cassandra.spark.cdc.IPartitionUpdateWrapper;
+import org.apache.cassandra.spark.data.partitioner.CassandraInstance;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class WatermarkerTests
+{
+    private static final String JOB_ID = "101";
+
+    private static IPartitionUpdateWrapper partitionUpdate(long timestamp)
+    {
+        IPartitionUpdateWrapper update = mock(IPartitionUpdateWrapper.class);
+        when(update.maxTimestampMicros()).thenReturn(timestamp * 1000L);  // In microseconds
+        return update;
+    }
+
+    @BeforeClass
+    public static void setup()
+    {
+        InMemoryWatermarker.TEST_THREAD_NAME = Thread.currentThread().getName();
+    }
+
+    @AfterClass
+    public static void tearDown()
+    {
+        InMemoryWatermarker.TEST_THREAD_NAME = null;
+    }
+
+    @Test
+    public void testHighwaterMark() throws ExecutionException, InterruptedException
+    {
+        Watermarker watermarker = InMemoryWatermarker.INSTANCE.instance(JOB_ID);
+        watermarker.clear();
+
+        assertEquals(watermarker, InMemoryWatermarker.INSTANCE.instance(JOB_ID));
+        InMemoryWatermarker.PartitionWatermarker partitionWatermarker = (InMemoryWatermarker.PartitionWatermarker) watermarker.instance(JOB_ID);
+        assertEquals(partitionWatermarker, partitionWatermarker.instance(JOB_ID));
+
+        // Calling from another thread should result in NPE
+        AtomicReference<Boolean> pass = new AtomicReference<>(false);
+        TestDataLayer.FILE_IO_EXECUTOR.submit(() -> {
+            try
+            {
+                InMemoryWatermarker.INSTANCE.instance(JOB_ID);
+                pass.set(false);
+            }
+            catch (NullPointerException exception)
+            {
+                pass.set(true);
+            }
+        }).get();
+        assertTrue(pass.get());
+
+        CassandraInstance in1 = new CassandraInstance("0L", "inst1", "DC1");
+        CassandraInstance in2 = new CassandraInstance("100L", "inst2", "DC1");
+
+        assertNull(watermarker.highWaterMark(in1));
+        assertNull(watermarker.highWaterMark(in2));
+
+        // Verify highwater mark tracks the highest seen
+        for (int index = 0; index <= 100; index++)
+        {
+            watermarker.updateHighWaterMark(new CommitLog.Marker(in1, 1L, 10 * index));
+        }
+        assertEquals(new CommitLog.Marker(in1, 1L, 1000), watermarker.highWaterMark(in1));
+        assertNull(watermarker.highWaterMark(in2));
+
+        watermarker.updateHighWaterMark(new CommitLog.Marker(in1, 2L, 1));
+        assertEquals(new CommitLog.Marker(in1, 2L, 1), watermarker.highWaterMark(in1));
+        for (int index = 0; index <= 100; index++)
+        {
+            watermarker.updateHighWaterMark(new CommitLog.Marker(in1, 2L, 5 * index));
+        }
+        assertEquals(new CommitLog.Marker(in1, 2L, 500), watermarker.highWaterMark(in1));
+
+        for (int index = 0; index <= 100; index++)
+        {
+            watermarker.updateHighWaterMark(new CommitLog.Marker(in1, 1L, 5 * index));
+        }
+        assertEquals(new CommitLog.Marker(in1, 2L, 500), watermarker.highWaterMark(in1));
+    }
+
+    @Test
+    public void testLateMutation()
+    {
+        Watermarker watermarker = InMemoryWatermarker.INSTANCE.instance(JOB_ID);
+        watermarker.clear();
+
+        CassandraInstance in1 = new CassandraInstance("0L", "inst1", "DC1");
+        for (int index = 0; index <= 100; index++)
+        {
+            watermarker.updateHighWaterMark(new CommitLog.Marker(in1, 2L, 5 * index));
+        }
+        for (int index = 0; index <= 100; index++)
+        {
+            watermarker.updateHighWaterMark(new CommitLog.Marker(in1, 10L, 5 * index));
+        }
+        CommitLog.Marker end = new CommitLog.Marker(in1, 10L, 500);
+        assertEquals(end, watermarker.highWaterMark(in1));
+
+        // Verify late mutations track earliest marker
+        long now = System.currentTimeMillis();
+        IPartitionUpdateWrapper update1 = partitionUpdate(now);
+        watermarker.recordReplicaCount(update1, 2);
+        assertEquals(end, watermarker.highWaterMark(in1));
+        IPartitionUpdateWrapper update2 = partitionUpdate(now);
+        watermarker.recordReplicaCount(update2, 2);
+        assertEquals(end, watermarker.highWaterMark(in1));
+        IPartitionUpdateWrapper update3 = partitionUpdate(now);
+        watermarker.recordReplicaCount(update3, 2);
+        assertEquals(end, watermarker.highWaterMark(in1));
+        IPartitionUpdateWrapper update4 = partitionUpdate(now);
+        watermarker.recordReplicaCount(update4, 2);
+
+        assertEquals(end, watermarker.highWaterMark(in1));
+        for (int index = 101; index <= 200; index++)
+        {
+            watermarker.updateHighWaterMark(new CommitLog.Marker(in1, 10L, 5 * index));
+        }
+        end = new CommitLog.Marker(in1, 10L, 1000);
+        assertEquals(end, watermarker.highWaterMark(in1));
+
+        assertTrue(watermarker.seenBefore(update1));
+        assertTrue(watermarker.seenBefore(update2));
+        assertTrue(watermarker.seenBefore(update3));
+        assertTrue(watermarker.seenBefore(update4));
+        assertEquals(2, watermarker.replicaCount(update1));
+        assertEquals(2, watermarker.replicaCount(update2));
+        assertEquals(2, watermarker.replicaCount(update3));
+        assertEquals(2, watermarker.replicaCount(update4));
+
+        // Clear mutations and verify watermark tracks last offset in order
+        watermarker.untrackReplicaCount(update2);
+        watermarker.untrackReplicaCount(update3);
+        watermarker.untrackReplicaCount(update4);
+        watermarker.untrackReplicaCount(update1);
+        assertEquals(end, watermarker.highWaterMark(in1));
+
+        assertEquals(0, watermarker.replicaCount(update1));
+        assertEquals(0, watermarker.replicaCount(update2));
+        assertEquals(0, watermarker.replicaCount(update3));
+        assertEquals(0, watermarker.replicaCount(update4));
+    }
+
+    @Test
+    public void testPublishedMutation()
+    {
+        Watermarker watermarker = InMemoryWatermarker.INSTANCE.instance(JOB_ID);
+        watermarker.clear();
+        CassandraInstance in1 = new CassandraInstance("0L", "inst1", "DC1");
+        long now = System.currentTimeMillis();
+        CommitLog.Marker end = new CommitLog.Marker(in1, 5L, 600);
+        watermarker.updateHighWaterMark(end);
+
+        IPartitionUpdateWrapper lateUpdate1 = partitionUpdate(now);
+        watermarker.recordReplicaCount(lateUpdate1, 2);
+        IPartitionUpdateWrapper lateUpdate2 = partitionUpdate(now);
+        watermarker.recordReplicaCount(lateUpdate2, 2);
+        IPartitionUpdateWrapper lateUpdate3 = partitionUpdate(now);
+        watermarker.recordReplicaCount(lateUpdate3, 2);
+
+        assertEquals(end, watermarker.highWaterMark(in1));
+
+        watermarker.untrackReplicaCount(lateUpdate1);
+        watermarker.untrackReplicaCount(lateUpdate2);
+        watermarker.untrackReplicaCount(lateUpdate3);
+
+        // Back at the highwater marker so published & late mutation markers have been cleared
+        assertEquals(end, watermarker.highWaterMark(in1));
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/CassandraDataSourceHelperCacheTest.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/CassandraDataSourceHelperCacheTest.java
new file mode 100644
index 0000000..4c344ec
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/CassandraDataSourceHelperCacheTest.java
@@ -0,0 +1,183 @@
+/*
+ * 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.cassandra.spark.data;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+import com.google.common.base.Ticker;
+import com.google.common.cache.Cache;
+import com.google.common.collect.ImmutableMap;
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.cassandra.clients.Sidecar;
+import org.apache.cassandra.clients.SslConfig;
+
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNotSame;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests the cache in {@link CassandraDataSourceHelper}
+ */
+public class CassandraDataSourceHelperCacheTest
+{
+    public static final Map<String, String> REQUIRED_CLIENT_CONFIG_OPTIONS =
+            ImmutableMap.of("sidecar_instances", "127.0.0.1",
+                            "keyspace", "big-data",
+                            "table", "customers");
+
+    private CacheTicker cacheTicker;
+
+    @Before
+    public void setup()
+    {
+        cacheTicker = new CacheTicker();
+        CassandraDataSourceHelper.initCassandraDataSourceCache(cacheTicker);
+    }
+
+    @Test
+    public void testCassandraDataLayerCacheExpiration() throws ExecutionException
+    {
+        Map<String, String> options = new HashMap<>(REQUIRED_CLIENT_CONFIG_OPTIONS);
+
+        Cache<Map<String, String>, CassandraDataLayer> cassandraDataLayerCache = CassandraDataSourceHelper.getCassandraDataLayerCache();
+        CassandraDataLayer dataLayer = getCassandraDataLayer(cassandraDataLayerCache, options, options);
+
+        assertNotNull(dataLayer);
+
+        // Advance ticker 1 minute
+        cacheTicker.advance(1, TimeUnit.MINUTES);
+
+        // Should get the same instance
+        CassandraDataLayer dataLayer1 = getCassandraDataLayer(cassandraDataLayerCache, options, options);
+        assertSame(dataLayer, dataLayer1);
+
+        // Advance ticker 1 hour
+        cacheTicker.advance(1, TimeUnit.HOURS);
+
+        // Should still get the same instance
+        CassandraDataLayer dataLayer2 = getCassandraDataLayer(cassandraDataLayerCache, options, options);
+        assertSame(dataLayer, dataLayer2);
+
+        // Advance ticker 10 hours and 58 minutes and 59 seconds
+        cacheTicker.advance(10, TimeUnit.HOURS);
+        cacheTicker.advance(58, TimeUnit.MINUTES);
+        cacheTicker.advance(59, TimeUnit.SECONDS);
+
+        // Should still get the same instance
+        CassandraDataLayer dataLayer3 = getCassandraDataLayer(cassandraDataLayerCache, options, options);
+        assertSame(dataLayer, dataLayer3);
+
+        // Advance ticker for 1 second
+        cacheTicker.advance(1, TimeUnit.SECONDS);
+
+        // Should get a different instance
+        CassandraDataLayer dataLayer4 = getCassandraDataLayer(cassandraDataLayerCache, options, options);
+        assertNotSame(dataLayer, dataLayer4);
+    }
+
+    @Test
+    public void testMultipleThreadsAccessingTheSameKey() throws InterruptedException, ExecutionException
+    {
+        Map<String, String> options = new HashMap<>(REQUIRED_CLIENT_CONFIG_OPTIONS);
+
+        Cache<Map<String, String>, CassandraDataLayer> cassandraDataLayerCache = CassandraDataSourceHelper.getCassandraDataLayerCache();
+
+        int threads = 20;
+        ExecutorService pool = Executors.newFixedThreadPool(threads);
+        CassandraDataLayer[] cassandraDataLayerArray = new CassandraDataLayer[threads];
+        CountDownLatch latch = new CountDownLatch(threads);
+
+        for (int thread = 0; thread < threads; thread++)
+        {
+            int finalI = thread;
+            pool.submit(() -> {
+                try
+                {
+                    // Invoke getCassandraDataLayer roughly at the same time
+                    latch.countDown();
+                    latch.await();
+                    // The first thread to win creates the object, the rest should get the same instance
+                    cassandraDataLayerArray[finalI] = getCassandraDataLayer(cassandraDataLayerCache, options, options);
+                }
+                catch (InterruptedException | ExecutionException exception)
+                {
+                    throw new RuntimeException(exception);
+                }
+            });
+        }
+
+        pool.shutdown();
+        assertTrue(pool.awaitTermination(1, TimeUnit.MINUTES));
+
+        for (int thread = 1; thread < threads; thread++)
+        {
+            assertSame(cassandraDataLayerArray[0], cassandraDataLayerArray[thread]);
+        }
+
+        // Advance ticker for 12 hours
+        cacheTicker.advance(12, TimeUnit.HOURS);
+        assertNotSame(cassandraDataLayerArray[0], getCassandraDataLayer(cassandraDataLayerCache, options, options));
+    }
+
+    private CassandraDataLayer getCassandraDataLayer(Cache<Map<String, String>, CassandraDataLayer> cassandraDataLayerCache,
+                                                     Map<String, String> key,
+                                                     Map<String, String> options) throws ExecutionException
+    {
+        return cassandraDataLayerCache.get(key, () -> {
+            CassandraDataLayer.ClientConfig config = CassandraDataLayer.ClientConfig.create(options);
+            return new CassandraDataLayer(config, Sidecar.ClientConfig.create(options), SslConfig.create(options));
+        });
+    }
+
+    private static class CacheTicker extends Ticker
+    {
+        private final AtomicLong nanos = new AtomicLong(0);
+
+        /**
+         * Returns the number of nanoseconds elapsed since this ticker's fixed point of reference
+         */
+        @Override
+        public long read()
+        {
+            return nanos.get();
+        }
+
+        /**
+         * Artificially advance time for a given {@code value} in the given {@link TimeUnit}
+         *
+         * @param value the value to advance
+         * @param unit  the {@link TimeUnit} for the given {@code value}
+         */
+        public void advance(long value, TimeUnit unit)
+        {
+            nanos.addAndGet(unit.toNanos(value));
+        }
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/CqlFieldComparators.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/CqlFieldComparators.java
new file mode 100644
index 0000000..830fba6
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/CqlFieldComparators.java
@@ -0,0 +1,260 @@
+/*
+ * 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.cassandra.spark.data;
+
+import java.math.BigDecimal;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.UUID;
+
+import org.junit.Test;
+
+import org.apache.cassandra.bridge.CassandraVersion;
+import org.apache.spark.sql.types.Decimal;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.quicktheories.QuickTheory.qt;
+import static org.quicktheories.generators.SourceDSL.integers;
+
+public class CqlFieldComparators extends VersionRunner
+{
+    public CqlFieldComparators(CassandraVersion version)
+    {
+        super(version);
+    }
+
+    private static CqlField createField(CqlField.CqlType type)
+    {
+        return new CqlField(false, false, false, "a", type, 0);
+    }
+
+    @Test
+    public void testStringComparator()
+    {
+        // ASCII
+        assertTrue(createField(bridge.ascii()).compare("a", "b") < 0);
+        assertEquals(0, createField(bridge.ascii()).compare("b", "b"));
+        assertTrue(createField(bridge.ascii()).compare("c", "b") > 0);
+        assertTrue(createField(bridge.ascii()).compare("b", "a") > 0);
+
+        assertTrue(createField(bridge.ascii()).compare("1", "2") < 0);
+        assertEquals(0, createField(bridge.ascii()).compare("2", "2"));
+        assertTrue(createField(bridge.ascii()).compare("3", "2") > 0);
+        assertTrue(createField(bridge.ascii()).compare("2", "1") > 0);
+
+        // TIMEUUID
+        assertTrue(createField(bridge.timeuuid()).compare("856f3600-8d57-11e9-9298-798dbb8bb043", "7a146960-8d57-11e9-94f8-1763d9f66f5e") < 0);
+        assertTrue(createField(bridge.timeuuid()).compare("964116b0-8d57-11e9-8097-5f40ae53943c", "8ebe0600-8d57-11e9-b507-7769fecef72d") > 0);
+        assertEquals(0, createField(bridge.timeuuid()).compare("9dda9590-8d57-11e9-9906-8b25b9c1ff19", "9dda9590-8d57-11e9-9906-8b25b9c1ff19"));
+
+        // UUID
+        UUID uuid1 = UUID.randomUUID();
+        UUID uuid2 = UUID.randomUUID();
+        UUID larger = uuid1.compareTo(uuid2) >= 0 ? uuid1 : uuid2;
+        UUID smaller = uuid1.compareTo(uuid2) <= 0 ? uuid1 : uuid2;
+        assertTrue(createField(bridge.uuid()).compare(smaller, larger) < 0);
+        assertTrue(createField(bridge.uuid()).compare(larger, smaller) > 0);
+        assertEquals(0, createField(bridge.uuid()).compare(smaller, smaller));
+        assertEquals(0, createField(bridge.uuid()).compare(larger, larger));
+
+        // TEXT
+        assertTrue(createField(bridge.text()).compare("abc", "abd") < 0);
+        assertTrue(createField(bridge.text()).compare("abd", "abc") > 0);
+        assertEquals(0, createField(bridge.text()).compare("abc", "abc"));
+        assertEquals(0, createField(bridge.text()).compare("abd", "abd"));
+
+        // VARCHAR
+        assertTrue(createField(bridge.varchar()).compare("abc", "abd") < 0);
+        assertTrue(createField(bridge.varchar()).compare("abd", "abc") > 0);
+        assertEquals(0, createField(bridge.varchar()).compare("abc", "abc"));
+        assertEquals(0, createField(bridge.varchar()).compare("abd", "abd"));
+    }
+
+    @Test
+    public void testBigDecimalComparator()
+    {
+        BigDecimal value = BigDecimal.valueOf(Long.MAX_VALUE).multiply(BigDecimal.valueOf(2));
+        Decimal decimal1 = Decimal.apply(value);
+        Decimal decimal2 = Decimal.apply(value.add(BigDecimal.ONE));
+        assertTrue(createField(bridge.decimal()).compare(decimal1, decimal2) < 0);
+        assertEquals(0, createField(bridge.decimal()).compare(decimal1, decimal1));
+        assertEquals(0, createField(bridge.decimal()).compare(decimal2, decimal2));
+        assertTrue(createField(bridge.decimal()).compare(decimal2, decimal1) > 0);
+    }
+
+    @Test
+    public void testVarIntComparator()
+    {
+        BigDecimal value = BigDecimal.valueOf(Long.MAX_VALUE).multiply(BigDecimal.valueOf(2));
+        Decimal decimal1 = Decimal.apply(value);
+        Decimal decimal2 = Decimal.apply(value.add(BigDecimal.ONE));
+        assertTrue(createField(bridge.varint()).compare(decimal1, decimal2) < 0);
+        assertEquals(0, createField(bridge.varint()).compare(decimal1, decimal1));
+        assertEquals(0, createField(bridge.varint()).compare(decimal2, decimal2));
+        assertTrue(createField(bridge.varint()).compare(decimal2, decimal1) > 0);
+    }
+
+    @Test
+    public void testIntegerComparator()
+    {
+        qt().forAll(integers().between(Integer.MIN_VALUE, Integer.MAX_VALUE - 1))
+            .checkAssert(integer -> {
+                assertTrue(createField(bridge.aInt()).compare(integer, integer + 1) < 0);
+                assertEquals(0, createField(bridge.aInt()).compare(integer, integer));
+                assertTrue(createField(bridge.aInt()).compare(integer + 1, integer) > 0);
+            });
+        assertEquals(0, createField(bridge.aInt()).compare(Integer.MAX_VALUE, Integer.MAX_VALUE));
+        assertEquals(0, createField(bridge.aInt()).compare(Integer.MIN_VALUE, Integer.MIN_VALUE));
+        assertTrue(createField(bridge.aInt()).compare(Integer.MIN_VALUE, Integer.MAX_VALUE) < 0);
+        assertTrue(createField(bridge.aInt()).compare(Integer.MAX_VALUE, Integer.MIN_VALUE) > 0);
+    }
+
+    @Test
+    public void testLongComparator()
+    {
+        assertTrue(createField(bridge.bigint()).compare(0L, 1L) < 0);
+        assertEquals(0, createField(bridge.bigint()).compare(1L, 1L));
+        assertTrue(createField(bridge.bigint()).compare(2L, 1L) > 0);
+        assertEquals(0, createField(bridge.bigint()).compare(Long.MAX_VALUE, Long.MAX_VALUE));
+        assertEquals(0, createField(bridge.bigint()).compare(Long.MIN_VALUE, Long.MIN_VALUE));
+        assertTrue(createField(bridge.bigint()).compare(Long.MIN_VALUE, Long.MAX_VALUE) < 0);
+        assertTrue(createField(bridge.bigint()).compare(Long.MAX_VALUE, Long.MIN_VALUE) > 0);
+    }
+
+    @Test
+    public void testTimeComparator()
+    {
+        assertTrue(createField(bridge.time()).compare(0L, 1L) < 0);
+        assertEquals(0, createField(bridge.time()).compare(1L, 1L));
+        assertTrue(createField(bridge.time()).compare(2L, 1L) > 0);
+        assertEquals(0, createField(bridge.time()).compare(Long.MAX_VALUE, Long.MAX_VALUE));
+        assertEquals(0, createField(bridge.time()).compare(Long.MIN_VALUE, Long.MIN_VALUE));
+        assertTrue(createField(bridge.time()).compare(Long.MIN_VALUE, Long.MAX_VALUE) < 0);
+        assertTrue(createField(bridge.time()).compare(Long.MAX_VALUE, Long.MIN_VALUE) > 0);
+    }
+
+    @Test
+    public void testBooleanComparator()
+    {
+        assertTrue(createField(bridge.bool()).compare(false, true) < 0);
+        assertEquals(0, createField(bridge.bool()).compare(false, false));
+        assertEquals(0, createField(bridge.bool()).compare(true, true));
+        assertTrue(createField(bridge.bool()).compare(true, false) > 0);
+    }
+
+    @Test
+    public void testFloatComparator()
+    {
+        assertTrue(createField(bridge.aFloat()).compare(1f, 2f) < 0);
+        assertEquals(0, createField(bridge.aFloat()).compare(2f, 2f));
+        assertTrue(createField(bridge.aFloat()).compare(2f, 1f) > 0);
+    }
+
+    @Test
+    public void testDoubleComparator()
+    {
+        assertTrue(createField(bridge.aDouble()).compare(1d, 2d) < 0);
+        assertEquals(0, createField(bridge.aDouble()).compare(2d, 2d));
+        assertTrue(createField(bridge.aDouble()).compare(2d, 1d) > 0);
+    }
+
+    @Test
+    public void testTimestampComparator()
+    {
+        long timestamp1 = 1L;
+        long timestamp2 = 2L;
+        assertTrue(createField(bridge.timestamp()).compare(timestamp1, timestamp2) < 0);
+        assertEquals(0, createField(bridge.timestamp()).compare(timestamp1, timestamp1));
+        assertEquals(0, createField(bridge.timestamp()).compare(timestamp2, timestamp2));
+        assertTrue(createField(bridge.timestamp()).compare(timestamp2, timestamp1) > 0);
+    }
+
+    @Test
+    public void testDateComparator()
+    {
+        int date1 = 1;
+        int date2 = 2;
+        assertTrue(createField(bridge.date()).compare(date1, date2) < 0);
+        assertEquals(0, createField(bridge.date()).compare(date1, date1));
+        assertEquals(0, createField(bridge.date()).compare(date2, date2));
+        assertTrue(createField(bridge.date()).compare(date2, date1) > 0);
+    }
+
+    @Test
+    public void testVoidComparator()
+    {
+        assertEquals(0, createField(bridge.empty()).compare(null, null));
+    }
+
+    @Test
+    public void testShortComparator()
+    {
+        assertTrue(createField(bridge.smallint()).compare((short) 1, (short) 2) < 0);
+        assertEquals(0, createField(bridge.smallint()).compare((short) 2, (short) 2));
+        assertTrue(createField(bridge.smallint()).compare((short) 2, (short) 1) > 0);
+    }
+
+    @Test
+    public void testByteArrayComparator()
+    {
+        byte[] bytes1 = new byte[]{0, 0, 0, 101};
+        byte[] bytes2 = new byte[]{0, 0, 0, 102};
+        byte[] bytes3 = new byte[]{0, 0, 1, 0};
+        byte[] bytes4 = new byte[]{1, 0, 0, 0};
+        assertTrue(createField(bridge.blob()).compare(bytes1, bytes2) < 0);
+        assertEquals(0, createField(bridge.blob()).compare(bytes1, bytes1));
+        assertEquals(0, createField(bridge.blob()).compare(bytes2, bytes2));
+        assertTrue(createField(bridge.blob()).compare(bytes2, bytes1) > 0);
+        assertTrue(createField(bridge.blob()).compare(bytes3, bytes1) > 0);
+        assertTrue(createField(bridge.blob()).compare(bytes3, bytes2) > 0);
+        assertTrue(createField(bridge.blob()).compare(bytes4, bytes3) > 0);
+    }
+
+    @Test
+    public void testInetComparator() throws UnknownHostException
+    {
+        byte[] ip1 = InetAddress.getByAddress(CqlFieldComparators.toByteArray(2130706433)).getAddress();  // 127.0.0.1
+        byte[] ip2 = InetAddress.getByAddress(CqlFieldComparators.toByteArray(2130706434)).getAddress();  // 127.0.0.2
+        assertTrue(createField(bridge.inet()).compare(ip1, ip2) < 0);
+        assertEquals(0, createField(bridge.inet()).compare(ip1, ip1));
+        assertEquals(0, createField(bridge.inet()).compare(ip2, ip2));
+        assertTrue(createField(bridge.inet()).compare(ip2, ip1) > 0);
+    }
+
+    private static byte[] toByteArray(int value)
+    {
+        return new byte[]{(byte) (value >> 24),
+                          (byte) (value >> 16),
+                          (byte) (value >>  8),
+                          (byte)  value};
+    }
+
+    @Test
+    public void testByteComparator()
+    {
+        byte byte1 = 101;
+        byte byte2 = 102;
+        assertTrue(createField(bridge.tinyint()).compare(byte1, byte2) < 0);
+        assertEquals(0, createField(bridge.tinyint()).compare(byte1, byte1));
+        assertEquals(0, createField(bridge.tinyint()).compare(byte2, byte2));
+        assertTrue(createField(bridge.tinyint()).compare(byte2, byte1) > 0);
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/CqlFieldTests.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/CqlFieldTests.java
new file mode 100644
index 0000000..b0bbb9e
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/CqlFieldTests.java
@@ -0,0 +1,256 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.cassandra.spark.data;
+
+import java.util.ArrayList;
+
+import org.junit.Test;
+
+import org.apache.cassandra.bridge.CassandraBridge;
+import org.apache.cassandra.bridge.CassandraVersion;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNotSame;
+import static org.junit.Assert.assertTrue;
+
+public class CqlFieldTests extends VersionRunner
+{
+    public CqlFieldTests(CassandraVersion version)
+    {
+        super(version);
+    }
+
+    @Test
+    public void testEquality()
+    {
+        CqlField field1 = new CqlField(true, false, false, "a", bridge.bigint(), 0);
+        CqlField field2 = new CqlField(true, false, false, "a", bridge.bigint(), 0);
+        assertNotSame(field1, field2);
+        assertEquals(field1, field2);
+        assertEquals(field1.hashCode(), field2.hashCode());
+        assertNotEquals(null, field1);
+        assertNotEquals(null, field2);
+        assertNotEquals(new ArrayList<>(), field1);
+        assertEquals(field1, field1);
+    }
+
+    @Test
+    public void testNotEqualsName()
+    {
+        CqlField field1 = new CqlField(true, false, false, "a", bridge.bigint(), 0);
+        CqlField field2 = new CqlField(true, false, false, "b", bridge.bigint(), 0);
+        assertNotSame(field1, field2);
+        assertNotEquals(field1, field2);
+        assertNotEquals(field1.hashCode(), field2.hashCode());
+    }
+
+    @Test
+    public void testNotEqualsType()
+    {
+        CqlField field1 = new CqlField(true, false, false, "a", bridge.bigint(), 0);
+        CqlField field2 = new CqlField(true, false, false, "a", bridge.timestamp(), 0);
+        assertNotSame(field1, field2);
+        assertNotEquals(field1, field2);
+        assertNotEquals(field1.hashCode(), field2.hashCode());
+    }
+
+    @Test
+    public void testNotEqualsKey()
+    {
+        CqlField field1 = new CqlField(true, false, false, "a", bridge.bigint(), 0);
+        CqlField field2 = new CqlField(false, true, false, "a", bridge.bigint(), 0);
+        assertNotSame(field1, field2);
+        assertNotEquals(field1, field2);
+        assertNotEquals(field1.hashCode(), field2.hashCode());
+    }
+
+    @Test
+    public void testNotEqualsPos()
+    {
+        CqlField field1 = new CqlField(true, false, false, "a", bridge.bigint(), 0);
+        CqlField field2 = new CqlField(true, false, false, "a", bridge.bigint(), 1);
+        assertNotSame(field1, field2);
+        assertNotEquals(field1, field2);
+        assertNotEquals(field1.hashCode(), field2.hashCode());
+    }
+
+    @Test
+    public void testCqlTypeParser()
+    {
+        testCqlTypeParser("set<text>", bridge.text());
+        testCqlTypeParser("set<float>", bridge.aFloat());
+        testCqlTypeParser("set<time>", bridge.time());
+        testCqlTypeParser("SET<BLOB>", bridge.blob());
+        testCqlTypeParser("list<ascii>", bridge.ascii());
+        testCqlTypeParser("list<int>", bridge.aInt());
+        testCqlTypeParser("LIST<BIGINT>", bridge.bigint());
+        testCqlTypeParser("map<int,text>", bridge.aInt(), bridge.text());
+        testCqlTypeParser("map<boolean , decimal>", bridge.bool(), bridge.decimal());
+        testCqlTypeParser("MAP<TIMEUUID,TIMESTAMP>", bridge.timeuuid(), bridge.timestamp());
+        testCqlTypeParser("MAP<VARCHAR , double>", bridge.varchar(), bridge.aDouble());
+        testCqlTypeParser("tuple<int, text>", bridge.aInt(), bridge.text());
+    }
+
+    @Test
+    public void testSplitMapTypes()
+    {
+        splitMap("", "", null);
+        splitMap("text", "text", null);
+        splitMap("bigint", "bigint", null);
+        splitMap("set<text>", "set<text>", null);
+        splitMap("text,bigint", "text", "bigint");
+        splitMap("varchar , float", "varchar", "float");
+        splitMap("varchar , float", "varchar", "float");
+        splitMap("date, frozen<set<text>>", "date", "frozen<set<text>>");
+        splitMap("timestamp, frozen<map<int, blob>>", "timestamp", "frozen<map<int, blob>>");
+        splitMap("frozen<list<timeuuid>>, frozen<map<uuid, double>>",
+                 "frozen<list<timeuuid>>", "frozen<map<uuid, double>>");
+        splitMap("frozen<map<int, float>>, frozen<map<blob, decimal>>",
+                 "frozen<map<int, float>>", "frozen<map<blob, decimal>>");
+        splitMap("frozen<map<int,float>>,frozen<map<blob,decimal>>",
+                 "frozen<map<int,float>>", "frozen<map<blob,decimal>>");
+        splitMap("text, frozen<map<text, set<text>>>", "text", "frozen<map<text, set<text>>>");
+        splitMap("frozen<map<set<int>,blob>>,   frozen<map<text, frozen<map<bigint, double>>>>",
+                 "frozen<map<set<int>,blob>>", "frozen<map<text, frozen<map<bigint, double>>>>");
+    }
+
+    @Test
+    public void testCqlNames()
+    {
+        assertEquals("set<bigint>", bridge.collection("set", bridge.bigint()).cqlName());
+        assertEquals("list<timestamp>", bridge.collection("LIST", bridge.timestamp()).cqlName());
+        assertEquals("map<text, int>", bridge.collection("Map", bridge.text(), bridge.aInt()).cqlName());
+        assertEquals("tuple<int, blob, varchar>",
+                     bridge.collection("tuple", bridge.aInt(), bridge.blob(), bridge.varchar()).cqlName());
+        assertEquals("tuple<int, blob, map<int, float>>",
+                     bridge.collection("tuPLe", bridge.aInt(), bridge.blob(), bridge.map(bridge.aInt(), bridge.aFloat())).cqlName());
+    }
+
+    @Test
+    public void testTuple()
+    {
+        String[] result = CassandraBridge.splitInnerTypes("a, b, c, d,e, f, g");
+        assertEquals("a", result[0]);
+        assertEquals("b", result[1]);
+        assertEquals("c", result[2]);
+        assertEquals("d", result[3]);
+        assertEquals("e", result[4]);
+        assertEquals("f", result[5]);
+        assertEquals("g", result[6]);
+    }
+
+    private static void splitMap(String str, String left, String right)
+    {
+        String[] result = CassandraBridge.splitInnerTypes(str);
+        if (left != null)
+        {
+            assertEquals(left, result[0]);
+        }
+        if (right != null)
+        {
+            assertEquals(right, result[1]);
+        }
+    }
+
+    @Test
+    public void testNestedSet()
+    {
+        CqlField.CqlType type = bridge.parseType("set<frozen<map<text, list<double>>>>");
+        assertNotNull(type);
+        assertEquals(type.internalType(), CqlField.CqlType.InternalType.Set);
+        CqlField.CqlType frozen = ((CqlField.CqlSet) type).type();
+        assertEquals(frozen.internalType(), CqlField.CqlType.InternalType.Frozen);
+        CqlField.CqlMap map = (CqlField.CqlMap) ((CqlField.CqlFrozen) frozen).inner();
+        assertEquals(map.keyType(), bridge.text());
+        assertEquals(map.valueType().internalType(), CqlField.CqlType.InternalType.List);
+        CqlField.CqlList list = (CqlField.CqlList) map.valueType();
+        assertEquals(list.type(), bridge.aDouble());
+    }
+
+    @Test
+    public void testFrozenCqlTypeParser()
+    {
+        CqlField.CqlType type = bridge.parseType("frozen<map<text, float>>");
+        assertNotNull(type);
+        assertEquals(type.internalType(), CqlField.CqlType.InternalType.Frozen);
+        CqlField.CqlType inner = ((CqlField.CqlFrozen) type).inner();
+        assertEquals(inner.internalType(), CqlField.CqlType.InternalType.Map);
+        CqlField.CqlMap map = (CqlField.CqlMap) inner;
+        assertEquals(map.keyType(), bridge.text());
+        assertEquals(map.valueType(), bridge.aFloat());
+    }
+
+    @Test
+    public void testFrozenCqlTypeNested()
+    {
+        CqlField.CqlType type = bridge.parseType("map<frozen<set<text>>, frozen<map<int, list<blob>>>>");
+        assertNotNull(type);
+        assertEquals(type.internalType(), CqlField.CqlType.InternalType.Map);
+
+        CqlField.CqlType key = ((CqlField.CqlMap) type).keyType();
+        assertEquals(key.internalType(), CqlField.CqlType.InternalType.Frozen);
+        CqlField.CqlCollection keyInner = (CqlField.CqlCollection) ((CqlField.CqlFrozen) key).inner();
+        assertEquals(keyInner.internalType(), CqlField.CqlType.InternalType.Set);
+        assertEquals(keyInner.type(), bridge.text());
+
+        CqlField.CqlType value = ((CqlField.CqlMap) type).valueType();
+        assertEquals(value.internalType(), CqlField.CqlType.InternalType.Frozen);
+        CqlField.CqlCollection valueInner = (CqlField.CqlCollection) ((CqlField.CqlFrozen) value).inner();
+        assertEquals(valueInner.internalType(), CqlField.CqlType.InternalType.Map);
+        CqlField.CqlMap valueMap = (CqlField.CqlMap) valueInner;
+        assertEquals(valueMap.keyType(), bridge.aInt());
+        assertEquals(valueMap.valueType().internalType(), CqlField.CqlType.InternalType.List);
+        assertEquals(((CqlField.CqlList) valueMap.valueType()).type(), bridge.blob());
+    }
+
+    private void testCqlTypeParser(String str, CqlField.CqlType expectedType)
+    {
+        testCqlTypeParser(str, expectedType, null);
+    }
+
+    private void testCqlTypeParser(String str, CqlField.CqlType expectedType, CqlField.CqlType otherType)
+    {
+        CqlField.CqlType type = bridge.parseType(str);
+        if (type instanceof CqlField.CqlTuple)
+        {
+            assertEquals(((CqlField.CqlTuple) type).type(0), expectedType);
+            if (otherType != null)
+            {
+                assertEquals(((CqlField.CqlTuple) type).type(1), otherType);
+            }
+        }
+        else if (type instanceof CqlField.CqlCollection)
+        {
+            assertEquals(((CqlField.CqlCollection) type).type(), expectedType);
+            if (otherType != null)
+            {
+                assertTrue(type instanceof CqlField.CqlMap);
+                assertEquals(((CqlField.CqlMap) type).valueType(), otherType);
+            }
+        }
+        else
+        {
+            assertTrue(type instanceof CqlField.NativeType);
+            assertEquals(type, expectedType);
+        }
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/DataLayerUnsupportedPushDownFiltersTest.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/DataLayerUnsupportedPushDownFiltersTest.java
new file mode 100644
index 0000000..ee6991d
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/DataLayerUnsupportedPushDownFiltersTest.java
@@ -0,0 +1,250 @@
+/*
+ * 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.cassandra.spark.data;
+
+import java.nio.file.Path;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.ImmutableList;
+import org.junit.Test;
+
+import org.apache.cassandra.bridge.CassandraBridge;
+import org.apache.cassandra.spark.TestDataLayer;
+import org.apache.cassandra.spark.utils.test.TestSchema;
+import org.apache.spark.sql.sources.And;
+import org.apache.spark.sql.sources.EqualNullSafe;
+import org.apache.spark.sql.sources.EqualTo;
+import org.apache.spark.sql.sources.Filter;
+import org.apache.spark.sql.sources.GreaterThan;
+import org.apache.spark.sql.sources.GreaterThanOrEqual;
+import org.apache.spark.sql.sources.In;
+import org.apache.spark.sql.sources.IsNotNull;
+import org.apache.spark.sql.sources.IsNull;
+import org.apache.spark.sql.sources.LessThan;
+import org.apache.spark.sql.sources.LessThanOrEqual;
+import org.apache.spark.sql.sources.Not;
+import org.apache.spark.sql.sources.Or;
+import org.apache.spark.sql.sources.StringContains;
+import org.apache.spark.sql.sources.StringEndsWith;
+import org.apache.spark.sql.sources.StringStartsWith;
+
+import static org.apache.cassandra.spark.TestUtils.getFileType;
+import static org.apache.cassandra.spark.TestUtils.runTest;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertSame;
+
+public class DataLayerUnsupportedPushDownFiltersTest
+{
+    @Test
+    public void testNoFilters()
+    {
+        runTest((partitioner, directory, bridge) -> {
+            TestSchema schema = TestSchema.basic(bridge);
+            List<Path> dataFiles = getFileType(directory, FileType.DATA).collect(Collectors.toList());
+            TestDataLayer dataLayer = new TestDataLayer(bridge, dataFiles, schema.buildTable());
+
+            Filter[] unsupportedFilters = dataLayer.unsupportedPushDownFilters(new Filter[0]);
+            assertNotNull(unsupportedFilters);
+            assertEquals(0, unsupportedFilters.length);
+        });
+    }
+
+    @Test
+    public void testSupportedEqualToFilter()
+    {
+        runTest((partitioner, directory, bridge) -> {
+            TestSchema schema = TestSchema.basic(bridge);
+            List<Path> dataFiles = getFileType(directory, FileType.DATA).collect(Collectors.toList());
+            TestDataLayer dataLayer = new TestDataLayer(bridge, dataFiles, schema.buildTable());
+
+            Filter[] allFilters = {new EqualTo("a", 5)};
+            Filter[] unsupportedFilters = dataLayer.unsupportedPushDownFilters(allFilters);
+            assertNotNull(unsupportedFilters);
+            // EqualTo is supported and 'a' is the partition key
+            assertEquals(0, unsupportedFilters.length);
+        });
+    }
+
+    @Test
+    public void testSupportedFilterCaseInsensitive()
+    {
+        runTest((partitioner, directory, bridge) -> {
+            TestSchema schema = TestSchema.basic(bridge);
+            List<Path> dataFiles = getFileType(directory, FileType.DATA).collect(Collectors.toList());
+            TestDataLayer dataLayer = new TestDataLayer(bridge, dataFiles, schema.buildTable());
+
+            Filter[] allFilters = {new EqualTo("A", 5)};
+            Filter[] unsupportedFilters = dataLayer.unsupportedPushDownFilters(allFilters);
+            assertNotNull(unsupportedFilters);
+            // EqualTo is supported and 'a' is the partition key
+            assertEquals(0, unsupportedFilters.length);
+        });
+    }
+
+    @Test
+    public void testSupportedInFilter()
+    {
+        runTest((partitioner, directory, bridge) -> {
+            TestSchema schema = TestSchema.basic(bridge);
+            List<Path> dataFiles = getFileType(directory, FileType.DATA).collect(Collectors.toList());
+            TestDataLayer dataLayer = new TestDataLayer(bridge, dataFiles, schema.buildTable());
+
+            Filter[] allFilters = {new In("a", new Object[]{5, 6, 7})};
+            Filter[] unsupportedFilters = dataLayer.unsupportedPushDownFilters(allFilters);
+            assertNotNull(unsupportedFilters);
+            // In is supported and 'a' is the partition key
+            assertEquals(0, unsupportedFilters.length);
+        });
+    }
+
+    @Test
+    public void testSupportedEqualFilterWithClusteringKey()
+    {
+        runTest((partitioner, directory, bridge) -> {
+            TestSchema schema = TestSchema.basic(bridge);
+            List<Path> dataFiles = getFileType(directory, FileType.DATA).collect(Collectors.toList());
+            TestDataLayer dataLayer = new TestDataLayer(bridge, dataFiles, schema.buildTable());
+
+            Filter[] allFilters = {new EqualTo("a", 5), new EqualTo("b", 8)};
+            Filter[] unsupportedFilters = dataLayer.unsupportedPushDownFilters(allFilters);
+            assertNotNull(unsupportedFilters);
+            // EqualTo is supported and 'a' is the partition key, the clustering key 'b' is not pushed down
+            assertEquals(1, unsupportedFilters.length);
+        });
+    }
+
+    @Test
+    public void testUnsupportedEqualFilterWithColumn()
+    {
+        runTest((partitioner, directory, bridge) -> {
+            TestSchema schema = TestSchema.basic(bridge);
+            List<Path> dataFiles = getFileType(directory, FileType.DATA).collect(Collectors.toList());
+            TestDataLayer dataLayer = new TestDataLayer(bridge, dataFiles, schema.buildTable());
+
+            EqualTo unsupportedNonPartitionKeyColumnFilter = new EqualTo("c", 25);
+            Filter[] allFilters = {new EqualTo("a", 5), unsupportedNonPartitionKeyColumnFilter};
+            Filter[] unsupportedFilters = dataLayer.unsupportedPushDownFilters(allFilters);
+            assertNotNull(unsupportedFilters);
+            // EqualTo is supported and 'a' is the partition key, 'c' is not supported
+            assertEquals(1, unsupportedFilters.length);
+            assertSame(unsupportedNonPartitionKeyColumnFilter, unsupportedFilters[0]);
+        });
+    }
+
+    @Test
+    public void testUnsupportedFilters()
+    {
+        runTest((partitioner, directory, bridge) -> {
+            TestSchema schema = TestSchema.basic(bridge);
+            List<Path> dataFiles = getFileType(directory, FileType.DATA).collect(Collectors.toList());
+            TestDataLayer dataLayer = new TestDataLayer(bridge, dataFiles, schema.buildTable());
+
+            List<Filter> unsupportedFilterList = ImmutableList.of(new EqualNullSafe("a", 5),
+                                                                  new GreaterThan("a", 5),
+                                                                  new GreaterThanOrEqual("a", 5),
+                                                                  new LessThan("a", 5),
+                                                                  new LessThanOrEqual("a", 5),
+                                                                  new IsNull("a"),
+                                                                  new IsNotNull("a"),
+                                                                  new And(new EqualTo("a", 5), new EqualTo("b", 6)),
+                                                                  new Or(new EqualTo("a", 5), new EqualTo("b", 6)),
+                                                                  new Not(new In("a", new Object[]{5, 6, 7})),
+                                                                  new StringStartsWith("a", "abc"),
+                                                                  new StringEndsWith("a", "abc"),
+                                                                  new StringContains("a", "abc"));
+
+            for (Filter unsupportedFilter : unsupportedFilterList)
+            {
+                Filter[] allFilters = {unsupportedFilter};
+                Filter[] unsupportedFilters = dataLayer.unsupportedPushDownFilters(allFilters);
+                assertNotNull(unsupportedFilters);
+                // Not supported
+                assertEquals(1, unsupportedFilters.length);
+            }
+        });
+    }
+
+    @Test
+    public void testSchemaWithCompositePartitionKey()
+    {
+        runTest((partitioner, directory, bridge) -> {
+            TestSchema schema = schemaWithCompositePartitionKey(bridge);
+            List<Path> dataFiles = getFileType(directory, FileType.DATA).collect(Collectors.toList());
+            TestDataLayer dataLayer = new TestDataLayer(bridge, dataFiles, schema.buildTable());
+
+            // a is part of a composite partition column
+            Filter[] allFilters = {new EqualTo("a", 5)};
+            Filter[] unsupportedFilters = dataLayer.unsupportedPushDownFilters(allFilters);
+            assertNotNull(unsupportedFilters);
+            // Filter push-down is disabled because not all partition columns are in the filter array
+            assertEquals(1, unsupportedFilters.length);
+
+            // a and b are part of a composite partition column
+            allFilters = new Filter[]{new EqualTo("a", 5), new EqualTo("b", 10)};
+            unsupportedFilters = dataLayer.unsupportedPushDownFilters(allFilters);
+            assertNotNull(unsupportedFilters);
+            // Filter push-down is disabled because not all partition columns are in the filter array
+            assertEquals(2, unsupportedFilters.length);
+
+            // a and b are part of a composite partition column, but d is not
+            allFilters = new Filter[]{new EqualTo("a", 5), new EqualTo("b", 10), new EqualTo("d", 20)};
+            unsupportedFilters = dataLayer.unsupportedPushDownFilters(allFilters);
+            assertNotNull(unsupportedFilters);
+            // Filter push-down is disabled because not all partition columns are in the filter array
+            assertEquals(3, unsupportedFilters.length);
+
+            // a and b are part of a composite partition column
+            allFilters = new Filter[]{new EqualTo("a", 5), new EqualTo("b", 10), new EqualTo("c", 15)};
+            unsupportedFilters = dataLayer.unsupportedPushDownFilters(allFilters);
+            assertNotNull(unsupportedFilters);
+            // Filter push-down is enabled because all the partition columns are part of the filter array
+            assertEquals(0, unsupportedFilters.length);
+        });
+    }
+
+    @Test
+    public void testDisablePushDownWhenPartitionKeyIsMissing()
+    {
+        runTest((partitioner, directory, bridge) -> {
+            TestSchema schema = TestSchema.basic(bridge);
+            List<Path> dataFiles = getFileType(directory, FileType.DATA).collect(Collectors.toList());
+            TestDataLayer dataLayer = new TestDataLayer(bridge, dataFiles, schema.buildTable());
+
+            // b is not the partition column
+            Filter[] allFilters = {new EqualTo("b", 25)};
+            Filter[] unsupportedFilters = dataLayer.unsupportedPushDownFilters(allFilters);
+            assertNotNull(unsupportedFilters);
+            // Filter push-down is disabled because the partition column is missing in the filters
+            assertEquals(1, unsupportedFilters.length);
+        });
+    }
+
+    private TestSchema schemaWithCompositePartitionKey(CassandraBridge bridge)
+    {
+        return TestSchema.builder()
+                         .withPartitionKey("a", bridge.aInt())
+                         .withPartitionKey("b", bridge.aInt())
+                         .withPartitionKey("c", bridge.aInt())
+                         .withClusteringKey("d", bridge.aInt())
+                         .withColumn("e", bridge.aInt()).build();
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/DefaultSizingTest.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/DefaultSizingTest.java
new file mode 100644
index 0000000..185286f
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/DefaultSizingTest.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.cassandra.spark.data;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Unit tests for the {@link DefaultSizing} class
+ */
+public class DefaultSizingTest
+{
+    @Test
+    public void testDefaultSizing()
+    {
+        Sizing sizing = new DefaultSizing(10);
+        assertEquals(10, sizing.getEffectiveNumberOfCores());
+
+        sizing = new DefaultSizing(100);
+        assertEquals(100, sizing.getEffectiveNumberOfCores());
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/LocalDataLayerTests.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/LocalDataLayerTests.java
new file mode 100644
index 0000000..dfe92f0
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/LocalDataLayerTests.java
@@ -0,0 +1,94 @@
+/*
+ * 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.cassandra.spark.data;
+
+import java.io.IOException;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.stream.Stream;
+
+import org.junit.Test;
+
+import org.apache.cassandra.bridge.CassandraVersion;
+import org.apache.cassandra.spark.data.partitioner.Partitioner;
+import org.apache.cassandra.spark.reader.SchemaTests;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNotSame;
+import static org.junit.Assert.assertTrue;
+
+public class LocalDataLayerTests extends VersionRunner
+{
+    public LocalDataLayerTests(CassandraVersion version)
+    {
+        super(version);
+    }
+
+    @Test
+    public void testLocalDataLayer() throws IOException
+    {
+        CassandraVersion version = bridge.getVersion();
+        Path directory1 = Files.createTempDirectory("d1");
+        Path directory2 = Files.createTempDirectory("d2");
+        Path directory3 = Files.createTempDirectory("d3");
+        Path directory4 = Files.createTempDirectory("d4");
+        LocalDataLayer dataLayer = new LocalDataLayer(version, "backup_test", SchemaTests.SCHEMA,
+                Stream.of(directory1, directory2, directory3, directory4)
+                      .map(directory -> directory.toAbsolutePath().toString())
+                      .toArray(String[]::new));
+        assertEquals(version, dataLayer.version());
+        assertEquals(1, dataLayer.partitionCount());
+        assertNotNull(dataLayer.cqlTable());
+        assertTrue(dataLayer.isInPartition(0, BigInteger.ZERO, ByteBuffer.wrap(new byte[0])));
+        assertEquals(Partitioner.Murmur3Partitioner, dataLayer.partitioner());
+        SSTablesSupplier ssTables = dataLayer.sstables(0, null, Collections.emptyList());
+        assertNotNull(ssTables);
+        assertTrue(ssTables.openAll((ssTable, isRepairPrimary) -> null).isEmpty());
+    }
+
+    @Test
+    public void testEquality()
+    {
+        CassandraVersion version = bridge.getVersion();
+        LocalDataLayer dataLayer1 = new LocalDataLayer(version, "backup_test", SchemaTests.SCHEMA,
+                "/var/lib/cassandra/data1/data/backup_test/sbr_test/snapshot/snapshotName/",
+                "/var/lib/cassandra/data2/data/backup_test/sbr_test/snapshot/snapshotName/",
+                "/var/lib/cassandra/data3/data/backup_test/sbr_test/snapshot/snapshotName/",
+                "/var/lib/cassandra/data4/data/backup_test/sbr_test/snapshot/snapshotName/");
+        LocalDataLayer dataLayer2 = new LocalDataLayer(version, "backup_test", SchemaTests.SCHEMA,
+                "/var/lib/cassandra/data1/data/backup_test/sbr_test/snapshot/snapshotName/",
+                "/var/lib/cassandra/data2/data/backup_test/sbr_test/snapshot/snapshotName/",
+                "/var/lib/cassandra/data3/data/backup_test/sbr_test/snapshot/snapshotName/",
+                "/var/lib/cassandra/data4/data/backup_test/sbr_test/snapshot/snapshotName/");
+        assertNotSame(dataLayer1, dataLayer2);
+        assertEquals(dataLayer1, dataLayer1);
+        assertEquals(dataLayer2, dataLayer2);
+        assertNotEquals(null, dataLayer2);
+        assertNotEquals(new ArrayList<>(), dataLayer1);
+        assertEquals(dataLayer1, dataLayer2);
+        assertEquals(dataLayer1.hashCode(), dataLayer2.hashCode());
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/PartitionedDataLayerTests.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/PartitionedDataLayerTests.java
new file mode 100644
index 0000000..f65698b
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/PartitionedDataLayerTests.java
@@ -0,0 +1,389 @@
+/*
+ * 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.cassandra.spark.data;
+
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Range;
+import org.apache.commons.lang3.RandomUtils;
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.cassandra.bridge.CassandraVersion;
+import org.apache.cassandra.spark.TestUtils;
+import org.apache.cassandra.spark.data.partitioner.CassandraInstance;
+import org.apache.cassandra.spark.data.partitioner.CassandraRing;
+import org.apache.cassandra.spark.data.partitioner.ConsistencyLevel;
+import org.apache.cassandra.spark.data.partitioner.JDKSerializationTests;
+import org.apache.cassandra.spark.data.partitioner.MultipleReplicasTests;
+import org.apache.cassandra.spark.data.partitioner.NotEnoughReplicasException;
+import org.apache.cassandra.spark.data.partitioner.Partitioner;
+import org.apache.cassandra.spark.data.partitioner.TokenPartitioner;
+import org.apache.cassandra.spark.reader.EmptyStreamScanner;
+import org.apache.cassandra.spark.reader.StreamScanner;
+import org.apache.cassandra.spark.sparksql.filters.PartitionKeyFilter;
+import org.apache.cassandra.spark.utils.test.TestSchema;
+import org.apache.spark.TaskContext;
+
+import static org.apache.cassandra.spark.data.PartitionedDataLayer.AvailabilityHint.AVAILABILITY_HINT_COMPARATOR;
+import static org.apache.cassandra.spark.data.PartitionedDataLayer.AvailabilityHint.DOWN;
+import static org.apache.cassandra.spark.data.PartitionedDataLayer.AvailabilityHint.JOINING;
+import static org.apache.cassandra.spark.data.PartitionedDataLayer.AvailabilityHint.LEAVING;
+import static org.apache.cassandra.spark.data.PartitionedDataLayer.AvailabilityHint.MOVING;
+import static org.apache.cassandra.spark.data.PartitionedDataLayer.AvailabilityHint.UNKNOWN;
+import static org.apache.cassandra.spark.data.PartitionedDataLayer.AvailabilityHint.UP;
+import static org.apache.cassandra.spark.data.partitioner.ConsistencyLevel.ALL;
+import static org.apache.cassandra.spark.data.partitioner.ConsistencyLevel.ANY;
+import static org.apache.cassandra.spark.data.partitioner.ConsistencyLevel.EACH_QUORUM;
+import static org.apache.cassandra.spark.data.partitioner.ConsistencyLevel.LOCAL_QUORUM;
+import static org.apache.cassandra.spark.data.partitioner.ConsistencyLevel.ONE;
+import static org.apache.cassandra.spark.data.partitioner.ConsistencyLevel.TWO;
+import static org.apache.cassandra.spark.data.partitioner.Partitioner.Murmur3Partitioner;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.CALLS_REAL_METHODS;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+import static org.quicktheories.QuickTheory.qt;
+import static org.quicktheories.generators.Generate.pick;
+
+public class PartitionedDataLayerTests extends VersionRunner
+{
+    int partitionId;
+
+    public PartitionedDataLayerTests(CassandraVersion version)
+    {
+        super(version);
+    }
+
+    @Before
+    public void setup()
+    {
+        partitionId = TaskContext.getPartitionId();
+    }
+
+    @Test
+    public void testSplitQuorumAllUp()
+    {
+        runSplitTests(1, UP);
+        runSplitTests(2, UP, UP);
+        runSplitTests(2, UP, UP, UP);
+        runSplitTests(3, UP, UP, UP, UP, UP);
+    }
+
+    @Test
+    public void testSplitQuorumOneDown()
+    {
+        runSplitTests(1, DOWN);
+        runSplitTests(2, DOWN, UP);
+        runSplitTests(2, DOWN, UP, UP);
+        runSplitTests(3, UP, DOWN, UP, UP, UP);
+    }
+
+    @Test
+    public void testSplitQuorumOneLeavingOrMoving()
+    {
+        runSplitTests(1, LEAVING);
+        runSplitTests(2, LEAVING, DOWN);
+        runSplitTests(2, DOWN, LEAVING, MOVING);
+        runSplitTests(3, UP, DOWN, UP, LEAVING, UP);
+    }
+
+    @Test
+    public void testSplitQuorumTwoDown()
+    {
+        runSplitTests(2, DOWN, DOWN);
+        runSplitTests(2, DOWN, UP, DOWN);
+        runSplitTests(3, UP, DOWN, UP, UP, DOWN);
+    }
+
+    @Test
+    public void testSplitAllWithLeavingAndMovingNodes()
+    {
+        runSplitTests(1, DOWN);
+        runSplitTests(1, UNKNOWN);
+        runSplitTests(3, UP, LEAVING, DOWN);
+        runSplitTests(5, UP, LEAVING, DOWN, JOINING, MOVING);
+    }
+
+    @Test
+    public void testParsingAvailabilityHint()
+    {
+        assertEquals(DOWN,    PartitionedDataLayer.AvailabilityHint.fromState("DOWN", "NORMAL"));
+        assertEquals(MOVING,  PartitionedDataLayer.AvailabilityHint.fromState("UP", "MOVING"));
+        assertEquals(LEAVING, PartitionedDataLayer.AvailabilityHint.fromState("UP", "LEAVING"));
+        assertEquals(UP,      PartitionedDataLayer.AvailabilityHint.fromState("UP", "NORMAL"));
+        assertEquals(UP,      PartitionedDataLayer.AvailabilityHint.fromState("UP", "STARTING"));
+        assertEquals(DOWN,    PartitionedDataLayer.AvailabilityHint.fromState("DOWN", "LEAVING"));
+        assertEquals(DOWN,    PartitionedDataLayer.AvailabilityHint.fromState("DOWN", "MOVING"));
+        assertEquals(DOWN,    PartitionedDataLayer.AvailabilityHint.fromState("DOWN", "NORMAL"));
+        assertEquals(UNKNOWN, PartitionedDataLayer.AvailabilityHint.fromState("UNKNOWN", "LEAVING"));
+        assertEquals(UNKNOWN, PartitionedDataLayer.AvailabilityHint.fromState("UNKNOWN", "MOVING"));
+        assertEquals(UNKNOWN, PartitionedDataLayer.AvailabilityHint.fromState("UNKNOWN", "NORMAL"));
+        assertEquals(JOINING, PartitionedDataLayer.AvailabilityHint.fromState("UP", "JOINING"));
+        assertEquals(UNKNOWN, PartitionedDataLayer.AvailabilityHint.fromState("randomState", "randomStatus"));
+    }
+
+    @Test
+    public void testAvailabilityHintComparator()
+    {
+        assertEquals(1,  AVAILABILITY_HINT_COMPARATOR.compare(UP, MOVING));
+        assertEquals(0,  AVAILABILITY_HINT_COMPARATOR.compare(LEAVING, MOVING));
+        assertEquals(-1, AVAILABILITY_HINT_COMPARATOR.compare(UNKNOWN, MOVING));
+        assertEquals(1,  AVAILABILITY_HINT_COMPARATOR.compare(LEAVING, UNKNOWN));
+        assertEquals(0,  AVAILABILITY_HINT_COMPARATOR.compare(DOWN, UNKNOWN));
+        assertEquals(0,  AVAILABILITY_HINT_COMPARATOR.compare(JOINING, DOWN));
+        assertEquals(1,  AVAILABILITY_HINT_COMPARATOR.compare(UP, DOWN));
+        assertEquals(-1, AVAILABILITY_HINT_COMPARATOR.compare(JOINING, UP));
+    }
+
+    @Test
+    public void testSplitAll()
+    {
+        runSplitTests(1, DOWN);
+        runSplitTests(1, UNKNOWN);
+        runSplitTests(3, UP, UP, DOWN);
+        runSplitTests(5, UP, UP, DOWN, UNKNOWN, UP);
+    }
+
+    @Test
+    public void testValidReplicationFactor()
+    {
+        PartitionedDataLayer.validateReplicationFactor(ANY,
+                                                       TestUtils.simpleStrategy(),
+                                                       null);
+        PartitionedDataLayer.validateReplicationFactor(ANY,
+                                                       TestUtils.networkTopologyStrategy(),
+                                                       null);
+        PartitionedDataLayer.validateReplicationFactor(ANY,
+                                                       TestUtils.networkTopologyStrategy(ImmutableMap.of("PV", 3)),
+                                                       null);
+        PartitionedDataLayer.validateReplicationFactor(ANY,
+                                                       TestUtils.networkTopologyStrategy(ImmutableMap.of("PV", 3)),
+                                                       "PV");
+        PartitionedDataLayer.validateReplicationFactor(LOCAL_QUORUM,
+                                                       TestUtils.networkTopologyStrategy(ImmutableMap.of("PV", 3)),
+                                                       "PV");
+        PartitionedDataLayer.validateReplicationFactor(ALL,
+                                                       TestUtils.networkTopologyStrategy(ImmutableMap.of("PV", 3, "MR", 3)),
+                                                       null);
+        PartitionedDataLayer.validateReplicationFactor(EACH_QUORUM,
+                                                       TestUtils.networkTopologyStrategy(ImmutableMap.of("PV", 3, "MR", 3)),
+                                                       null);
+        PartitionedDataLayer.validateReplicationFactor(ANY,
+                                                       TestUtils.networkTopologyStrategy(ImmutableMap.of("PV", 3, "MR", 3)),
+                                                       null);
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void testReplicationFactorDCRequired()
+    {
+        // DC required for DC-local consistency level
+        PartitionedDataLayer.validateReplicationFactor(LOCAL_QUORUM,
+                                                       TestUtils.networkTopologyStrategy(ImmutableMap.of("PV", 3, "MR", 3)),
+                                                       null);
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void testReplicationFactorUnknownDC()
+    {
+        PartitionedDataLayer.validateReplicationFactor(LOCAL_QUORUM,
+                                                       TestUtils.networkTopologyStrategy(ImmutableMap.of("PV", 3, "MR", 3)),
+                                                       "ST");
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void testReplicationFactorRF0()
+    {
+        PartitionedDataLayer.validateReplicationFactor(LOCAL_QUORUM,
+                                                       TestUtils.networkTopologyStrategy(ImmutableMap.of("PV", 3, "MR", 0)),
+                                                       "MR");
+    }
+
+    @Test
+    public void testSSTableSupplier()
+    {
+        CassandraRing ring = TestUtils.createRing(Murmur3Partitioner, 3);
+        CqlTable table = TestSchema.basic(bridge).buildTable();
+        DataLayer dataLayer = new JDKSerializationTests.TestPartitionedDataLayer(bridge, 4, 32, null, ring, table);
+        SSTablesSupplier supplier = dataLayer.sstables(partitionId, null, new ArrayList<>());
+        Set<MultipleReplicasTests.TestSSTableReader> ssTableReaders =
+                supplier.openAll((ssTable, isRepairPrimary) -> new MultipleReplicasTests.TestSSTableReader(ssTable));
+        assertNotNull(ssTableReaders);
+    }
+
+    @Test
+    public void testSSTableSupplierWithMatchingFilters()
+    {
+        CassandraRing ring = TestUtils.createRing(Partitioner.Murmur3Partitioner, 3);
+        CqlTable table = TestSchema.basic(bridge).buildTable();
+        DataLayer dataLayer = new JDKSerializationTests.TestPartitionedDataLayer(bridge, 4, 32, null, ring, table);
+
+        PartitionKeyFilter filter = PartitionKeyFilter.create(ByteBuffer.wrap(RandomUtils.nextBytes(10)),
+                                                              BigInteger.valueOf(-9223372036854775808L));
+        SSTablesSupplier supplier = dataLayer.sstables(partitionId, null, Collections.singletonList(filter));
+        Set<MultipleReplicasTests.TestSSTableReader> ssTableReaders =
+                supplier.openAll((ssTable, isRepairPrimary) -> new MultipleReplicasTests.TestSSTableReader(ssTable));
+        assertNotNull(ssTableReaders);
+    }
+
+    @Test(expected = NotEnoughReplicasException.class)
+    public void testSSTableSupplierWithNonMatchingFilters()
+    {
+        CassandraRing ring = TestUtils.createRing(Partitioner.Murmur3Partitioner, 3);
+        CqlTable table = TestSchema.basic(bridge).buildTable();
+        DataLayer dataLayer = new JDKSerializationTests.TestPartitionedDataLayer(bridge, 4, 32, null, ring, table);
+
+        PartitionKeyFilter filter = PartitionKeyFilter.create(ByteBuffer.wrap(RandomUtils.nextBytes(10)),
+                                                              BigInteger.valueOf(6917529027641081853L));
+        SSTablesSupplier supplier = dataLayer.sstables(partitionId, null, Collections.singletonList(filter));
+    }
+
+    @Test
+    public void testFiltersInRange() throws Exception
+    {
+        Map<Integer, Range<BigInteger>> reversePartitionMap = Collections.singletonMap(
+                TaskContext.getPartitionId(), Range.closed(BigInteger.ONE, BigInteger.valueOf(2L)));
+        TokenPartitioner mockPartitioner = mock(TokenPartitioner.class);
+        when(mockPartitioner.reversePartitionMap()).thenReturn(reversePartitionMap);
+
+        PartitionedDataLayer dataLayer = mock(PartitionedDataLayer.class, CALLS_REAL_METHODS);
+        when(dataLayer.tokenPartitioner()).thenReturn(mockPartitioner);
+
+        PartitionKeyFilter filterInRange = PartitionKeyFilter.create(ByteBuffer.wrap(new byte[10]),
+                                                                     BigInteger.valueOf(2L));
+        PartitionKeyFilter filterOutsideRange = PartitionKeyFilter.create(ByteBuffer.wrap(new byte[10]),
+                                                                          BigInteger.TEN);
+        PartitionKeyFilter randomFilter = mock(PartitionKeyFilter.class);
+        when(randomFilter.overlaps(any())).thenReturn(true);
+
+        assertFalse(dataLayer.partitionKeyFiltersInRange(partitionId,
+                                                         Collections.singletonList(randomFilter)).isEmpty());
+        assertEquals(2, dataLayer.partitionKeyFiltersInRange(partitionId,
+                                                             Arrays.asList(filterInRange, randomFilter)).size());
+        assertEquals(2, dataLayer.partitionKeyFiltersInRange(partitionId,
+                                                             Arrays.asList(filterInRange, filterOutsideRange, randomFilter)).size());
+
+        // Filter does not fall in spark token range
+        StreamScanner scanner = dataLayer.openCompactionScanner(partitionId,
+                                                                Collections.singletonList(filterOutsideRange));
+        assertTrue(scanner instanceof EmptyStreamScanner);
+    }
+
+    @SuppressWarnings("UnstableApiUsage")
+    private static void runSplitTests(int minReplicas, PartitionedDataLayer.AvailabilityHint... availabilityHint)
+    {
+        int numInstances = availabilityHint.length;
+        TestUtils.runTest((partitioner, dir, bridge) -> {
+            CassandraRing ring = TestUtils.createRing(partitioner, numInstances);
+            List<CassandraInstance> instances = new ArrayList<>(ring.instances());
+            instances.sort(Comparator.comparing(CassandraInstance::nodeName));
+            TokenPartitioner tokenPartitioner = new TokenPartitioner(ring, 1, 32);
+            Map<CassandraInstance, PartitionedDataLayer.AvailabilityHint> availableMap = new HashMap<>(numInstances);
+            for (int instance = 0; instance < numInstances; instance++)
+            {
+                availableMap.put(instances.get(instance), availabilityHint[instance]);
+            }
+
+            Map<Range<BigInteger>, List<CassandraInstance>> ranges =
+                    ring.getSubRanges(tokenPartitioner.getTokenRange(0)).asMapOfRanges();
+            PartitionedDataLayer.ReplicaSet replicaSet =
+                    PartitionedDataLayer.splitReplicas(instances, ranges, availableMap::get, minReplicas, 0);
+            assertEquals(minReplicas, replicaSet.primary().size());
+            assertEquals(numInstances - minReplicas, replicaSet.backup().size());
+
+            List<CassandraInstance> sortedInstances = new ArrayList<>(instances);
+            sortedInstances.sort(Comparator.comparing(availableMap::get, AVAILABILITY_HINT_COMPARATOR));
+            for (int instance = 0; instance < sortedInstances.size(); instance++)
+            {
+                if (instance < minReplicas)
+                {
+                    assertTrue(replicaSet.primary().contains(sortedInstances.get(instance)));
+                }
+                else
+                {
+                    assertTrue(replicaSet.backup().contains(sortedInstances.get(instance)));
+                }
+            }
+        });
+    }
+
+    @Test
+    public void testSplitReplicas()
+    {
+        ReplicationFactor replicationFactor = TestUtils.networkTopologyStrategy();
+        TestUtils.runTest((partitioner, dir, bridge) ->
+                qt().forAll(pick(Arrays.asList(3, 32, 1024)),
+                            pick(Arrays.asList(LOCAL_QUORUM, ONE, ALL, TWO)),
+                            pick(Arrays.asList(1, 32, 1024)),
+                            pick(Arrays.asList(1, 32, 1024)))
+                    .checkAssert((numInstances, consistencyLevel, numCores, defaultParallelism) ->
+                          PartitionedDataLayerTests.testSplitReplicas(TestUtils.createRing(partitioner, numInstances),
+                                                                      consistencyLevel,
+                                                                      defaultParallelism,
+                                                                      numCores,
+                                                                      replicationFactor,
+                                                                      "DC1")));
+    }
+
+    @SuppressWarnings("UnstableApiUsage")
+    private static void testSplitReplicas(CassandraRing ring,
+                                          ConsistencyLevel consistencyLevel,
+                                          int defaultParallelism,
+                                          int numCores,
+                                          ReplicationFactor replicationFactor,
+                                          String dc)
+    {
+        TokenPartitioner tokenPartitioner = new TokenPartitioner(ring, defaultParallelism, numCores);
+
+        for (int partition = 0; partition < tokenPartitioner.numPartitions(); partition++)
+        {
+            Range<BigInteger> range = tokenPartitioner.getTokenRange(partition);
+            Map<Range<BigInteger>, List<CassandraInstance>> subRanges = ring.getSubRanges(range).asMapOfRanges();
+            Set<CassandraInstance> replicas = PartitionedDataLayer.rangesToReplicas(consistencyLevel, dc, subRanges);
+            Function<CassandraInstance, PartitionedDataLayer.AvailabilityHint> availability = instances -> UP;
+            int minReplicas = consistencyLevel.blockFor(replicationFactor, dc);
+            PartitionedDataLayer.ReplicaSet replicaSet = PartitionedDataLayer.splitReplicas(consistencyLevel,
+                                                                                            dc,
+                                                                                            subRanges,
+                                                                                            replicas,
+                                                                                            availability,
+                                                                                            minReplicas,
+                                                                                            0);
+            assertNotNull(replicaSet);
+            assertTrue(Collections.disjoint(replicaSet.primary(), replicaSet.backup()));
+            assertEquals(replicas.size(), replicaSet.primary().size() + replicaSet.backup().size());
+        }
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/ReplicationFactorTests.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/ReplicationFactorTests.java
new file mode 100644
index 0000000..82d9d88
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/ReplicationFactorTests.java
@@ -0,0 +1,113 @@
+/*
+ * 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.cassandra.spark.data;
+
+import java.util.ArrayList;
+
+import com.google.common.collect.ImmutableMap;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotSame;
+
+public class ReplicationFactorTests
+{
+    @Test
+    public void testReplicationFactorNtsClassNameOnly()
+    {
+        ReplicationFactor replicationFactor = new ReplicationFactor(ImmutableMap.of(
+                "class", "NetworkTopologyStrategy",
+                "datacenter1", "3",
+                "datacenter2", "5"));
+        assertEquals(ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy,
+                     replicationFactor.getReplicationStrategy());
+        assertEquals(Integer.valueOf(3), replicationFactor.getOptions().get("datacenter1"));
+        assertEquals(Integer.valueOf(5), replicationFactor.getOptions().get("datacenter2"));
+    }
+
+    @Test
+    public void testReplicationFactorNtsFullyQualifiedClassName()
+    {
+        ReplicationFactor replicationFactor = new ReplicationFactor(ImmutableMap.of(
+                "class", "org.apache.cassandra.locator.NetworkTopologyStrategy",
+                "datacenter1", "9",
+                "datacenter2", "2"));
+        assertEquals(ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy,
+                     replicationFactor.getReplicationStrategy());
+        assertEquals(Integer.valueOf(9), replicationFactor.getOptions().get("datacenter1"));
+        assertEquals(Integer.valueOf(2), replicationFactor.getOptions().get("datacenter2"));
+    }
+
+    @Test
+    public void testReplicationFactorSimpleClassNameOnly()
+    {
+        ReplicationFactor replicationFactor = new ReplicationFactor(ImmutableMap.of(
+                "class", "SimpleStrategy",
+                "replication_factor", "3"));
+        assertEquals(ReplicationFactor.ReplicationStrategy.SimpleStrategy, replicationFactor.getReplicationStrategy());
+        assertEquals(Integer.valueOf(3), replicationFactor.getOptions().get("replication_factor"));
+    }
+
+    @Test
+    public void testReplicationFactorSimpleFullyQualifiedClassName()
+    {
+        ReplicationFactor replicationFactor = new ReplicationFactor(ImmutableMap.of(
+                "class", "org.apache.cassandra.locator.SimpleStrategy",
+                "replication_factor", "5"));
+        assertEquals(ReplicationFactor.ReplicationStrategy.SimpleStrategy, replicationFactor.getReplicationStrategy());
+        assertEquals(Integer.valueOf(5), replicationFactor.getOptions().get("replication_factor"));
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void testUnexpectedRFClass()
+    {
+        new ReplicationFactor(ImmutableMap.of(
+                "class", "org.apache.cassandra.locator.NotSimpleStrategy",
+                "replication_factor", "5"));
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void testUnknownRFClass()
+    {
+        new ReplicationFactor(ImmutableMap.of(
+                "class", "NoSuchStrategy",
+                "replication_factor", "5"));
+    }
+
+    @Test
+    public void testEquality()
+    {
+        ReplicationFactor replicationFactor1 = new ReplicationFactor(ImmutableMap.of(
+                "class", "org.apache.cassandra.locator.SimpleStrategy",
+                "replication_factor", "5"));
+        ReplicationFactor replicationFactor2 = new ReplicationFactor(ImmutableMap.of(
+                "class", "org.apache.cassandra.locator.SimpleStrategy",
+                "replication_factor", "5"));
+        assertNotSame(replicationFactor1, replicationFactor2);
+        assertNotEquals(null, replicationFactor1);
+        assertNotEquals(replicationFactor2, null);
+        assertEquals(replicationFactor1, replicationFactor1);
+        assertEquals(replicationFactor2, replicationFactor2);
+        assertNotEquals(new ArrayList<>(), replicationFactor1);
+        assertEquals(replicationFactor1, replicationFactor2);
+        assertEquals(replicationFactor1.hashCode(), replicationFactor2.hashCode());
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/VersionRunner.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/VersionRunner.java
new file mode 100644
index 0000000..c493466
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/VersionRunner.java
@@ -0,0 +1,54 @@
+/*
+ * 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.cassandra.spark.data;
+
+import java.util.Collection;
+
+import com.google.common.collect.ImmutableList;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.cassandra.bridge.CassandraBridge;
+import org.apache.cassandra.bridge.CassandraBridgeFactory;
+import org.apache.cassandra.bridge.CassandraVersion;
+
+/**
+ * Run tests Parameterized for multiple versions of Cassandra
+ */
+// TODO: Merge org.apache.cassandra.bridge.VersionRunner and org.apache.cassandra.spark.data.VersionRunner
+@RunWith(Parameterized.class)
+public abstract class VersionRunner
+{
+    protected final CassandraVersion version;
+    protected final CassandraBridge bridge;
+
+    @Parameterized.Parameters
+    public static Collection<Object[]> versions()
+    {
+        // TODO: Make use of TestUtils.testableVersions() instead
+        return ImmutableList.of(new Object[]{CassandraVersion.FOURZERO});
+    }
+
+    public VersionRunner(CassandraVersion version)
+    {
+        this.version = version;
+        this.bridge = CassandraBridgeFactory.get(version);
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/partitioner/CassandraRingTests.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/partitioner/CassandraRingTests.java
new file mode 100644
index 0000000..670ca32
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/partitioner/CassandraRingTests.java
@@ -0,0 +1,446 @@
+/*
+ * 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.cassandra.spark.data.partitioner;
+
+import java.math.BigInteger;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Multimap;
+import com.google.common.collect.Range;
+import com.google.common.collect.RangeSet;
+import com.google.common.collect.TreeRangeSet;
+import org.junit.Test;
+
+import org.apache.cassandra.spark.data.ReplicationFactor;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+@SuppressWarnings("UnstableApiUsage")
+public class CassandraRingTests
+{
+    private static Range<BigInteger> mergeRanges(Collection<Range<BigInteger>> ranges)
+    {
+        Range<BigInteger> mergedRange = Range.closedOpen(BigInteger.ZERO, BigInteger.ZERO);
+        for (Range<BigInteger> range : ranges)
+        {
+            mergedRange = mergedRange.span(range);
+        }
+
+        return mergedRange;
+    }
+
+    private void validateRanges(Collection<Range<BigInteger>> ranges,
+                                Collection<BigInteger> validTokens,
+                                Collection<BigInteger> invalidTokens)
+    {
+        RangeSet<BigInteger> rangeSet = TreeRangeSet.create();
+
+        ranges.forEach(rangeSet::add);
+        validTokens.forEach(token -> assertTrue(token + " should have been a valid token", rangeSet.contains(token)));
+        invalidTokens.forEach(token -> assertFalse(rangeSet.contains(token)));
+    }
+
+    @Test
+    public void testSimpleStrategyRF3()
+    {
+        List<CassandraInstance> instances = Arrays.asList(new CassandraInstance("0", "local0-i1", "DEV"),
+                                                          new CassandraInstance("100", "local0-i2", "DEV"),
+                                                          new CassandraInstance("200", "local0-i3", "DEV"));
+        CassandraRing ring = new CassandraRing(Partitioner.Murmur3Partitioner,
+                                               "test",
+                                               new ReplicationFactor(ImmutableMap.of(
+                                                       "class", "org.apache.cassandra.locator.SimpleStrategy",
+                                                       "replication_factor", "3")),
+                                               instances);
+
+        assertArrayEquals(ring.tokens().toArray(), Arrays.asList(BigInteger.valueOf(0L),
+                                                                 BigInteger.valueOf(100L),
+                                                                 BigInteger.valueOf(200L)).toArray());
+
+        Multimap<CassandraInstance, Range<BigInteger>> tokenRanges = ring.tokenRanges();
+        for (CassandraInstance instance : instances)
+        {
+            assertEquals(mergeRanges(tokenRanges.get(instance)),
+                         Range.closed(Partitioner.Murmur3Partitioner.minToken(),
+                                      Partitioner.Murmur3Partitioner.maxToken()));
+        }
+    }
+
+    @Test
+    public void testSimpleStrategyRF1()
+    {
+        List<CassandraInstance> instances = Arrays.asList(new CassandraInstance("0", "local0-i1", "DEV"),
+                                                          new CassandraInstance("100", "local0-i2", "DEV"),
+                                                          new CassandraInstance("200", "local0-i3", "DEV"));
+        CassandraRing ring = new CassandraRing(Partitioner.Murmur3Partitioner,
+                                               "test",
+                                               new ReplicationFactor(ImmutableMap.of(
+                                                       "class", "org.apache.cassandra.locator.SimpleStrategy",
+                                                       "replication_factor", "1")),
+                                               instances);
+
+        assertArrayEquals(ring.tokens().toArray(), Arrays.asList(BigInteger.valueOf(0L),
+                                                                 BigInteger.valueOf(100L),
+                                                                 BigInteger.valueOf(200L)).toArray());
+
+        Multimap<CassandraInstance, Range<BigInteger>> tokenRanges = ring.tokenRanges();
+
+        // token(0) => [201 - 0] => [201 - MAX], [MIN - 0]
+        validateRanges(tokenRanges.get(instances.get(0)),
+                       Arrays.asList(BigInteger.ZERO,
+                                     Partitioner.Murmur3Partitioner.minToken(),
+                                     Partitioner.Murmur3Partitioner.maxToken(),
+                                     BigInteger.valueOf(201L)),
+                       Arrays.asList(BigInteger.valueOf(200L),
+                                     BigInteger.valueOf(100L),
+                                     BigInteger.valueOf(1L)));
+
+        // token(100) => [1 - 100]
+        validateRanges(tokenRanges.get(instances.get(1)),
+                       Arrays.asList(BigInteger.valueOf(1L),
+                                     BigInteger.valueOf(100L),
+                                     BigInteger.valueOf(50L)),
+                       Arrays.asList(BigInteger.valueOf(101L),
+                                     BigInteger.valueOf(200L),
+                                     BigInteger.valueOf(0L),
+                                     Partitioner.Murmur3Partitioner.minToken(),
+                                     Partitioner.Murmur3Partitioner.maxToken()));
+
+        // token(200) => [101 - 200]
+        validateRanges(tokenRanges.get(instances.get(2)),
+                       Arrays.asList(BigInteger.valueOf(101L),
+                                     BigInteger.valueOf(150L),
+                                     BigInteger.valueOf(200L)),
+                       Arrays.asList(BigInteger.valueOf(100L),
+                                     BigInteger.valueOf(201L),
+                                     BigInteger.valueOf(1L),
+                                     Partitioner.Murmur3Partitioner.minToken(),
+                                     Partitioner.Murmur3Partitioner.maxToken()));
+    }
+
+    @Test
+    public void testSimpleStrategyRF2()
+    {
+        List<CassandraInstance> instances = Arrays.asList(new CassandraInstance("0", "local0-i1", "DEV"),
+                                                          new CassandraInstance("100", "local0-i2", "DEV"),
+                                                          new CassandraInstance("200", "local0-i3", "DEV"));
+        CassandraRing ring = new CassandraRing(Partitioner.Murmur3Partitioner,
+                                               "test",
+                                               new ReplicationFactor(ImmutableMap.of(
+                                                       "class", "org.apache.cassandra.locator.SimpleStrategy",
+                                                       "replication_factor", "2")),
+                                               instances);
+
+        assertArrayEquals(ring.tokens().toArray(), Arrays.asList(BigInteger.valueOf(0L),
+                                                                 BigInteger.valueOf(100L),
+                                                                 BigInteger.valueOf(200L)).toArray());
+
+        Multimap<CassandraInstance, Range<BigInteger>> tokenRanges = ring.tokenRanges();
+
+        // token(0) => [101 - 0] => [101 - MAX] [MIN - 0]
+        validateRanges(tokenRanges.get(instances.get(0)),
+                       Arrays.asList(BigInteger.ZERO,
+                                     Partitioner.Murmur3Partitioner.minToken(),
+                                     Partitioner.Murmur3Partitioner.maxToken(),
+                                     BigInteger.valueOf(200L),
+                                     BigInteger.valueOf(101L)),
+                       Arrays.asList(BigInteger.valueOf(100L),
+                                     BigInteger.valueOf(1L)));
+
+        // token(100) => [201 - 100] => [201 - MAX] [MIN - 100]
+        validateRanges(tokenRanges.get(instances.get(1)),
+                       Arrays.asList(BigInteger.valueOf(0L),
+                                     Partitioner.Murmur3Partitioner.minToken(),
+                                     Partitioner.Murmur3Partitioner.maxToken(),
+                                     BigInteger.valueOf(100L),
+                                     BigInteger.valueOf(201L)),
+                       Arrays.asList(BigInteger.valueOf(101L),
+                                     BigInteger.valueOf(200L)));
+
+        // token(200) => [1 - 200]
+        validateRanges(tokenRanges.get(instances.get(2)),
+                       Arrays.asList(BigInteger.valueOf(1L),
+                                     BigInteger.valueOf(100L),
+                                     BigInteger.valueOf(200L)),
+                       Arrays.asList(BigInteger.valueOf(0L),
+                                     BigInteger.valueOf(201L),
+                                     Partitioner.Murmur3Partitioner.minToken(),
+                                     Partitioner.Murmur3Partitioner.maxToken()));
+    }
+
+    @Test
+    public void testNetworkStrategyRF33()
+    {
+        List<CassandraInstance> instances = Arrays.asList(new CassandraInstance("0", "local0-i1", "DC1"),
+                                                          new CassandraInstance("100", "local0-i2", "DC1"),
+                                                          new CassandraInstance("200", "local0-i3", "DC1"),
+                                                          new CassandraInstance("1", "local1-i1", "DC2"),
+                                                          new CassandraInstance("101", "local1-i2", "DC2"),
+                                                          new CassandraInstance("201", "local1-i3", "DC2"));
+
+        CassandraRing ring = new CassandraRing(
+                Partitioner.Murmur3Partitioner,
+                "test",
+                new ReplicationFactor(ImmutableMap.of("class", "org.apache.cassandra.locator.NetworkTopologyStrategy",
+                                                      "DC1", "3",
+                                                      "DC2", "3")),
+                Arrays.asList(new CassandraInstance("0", "local0-i1", "DC1"),
+                              new CassandraInstance("100", "local0-i2", "DC1"),
+                              new CassandraInstance("200", "local0-i3", "DC1"),
+                              new CassandraInstance("1", "local1-i1", "DC2"),
+                              new CassandraInstance("101", "local1-i2", "DC2"),
+                              new CassandraInstance("201", "local1-i3", "DC2")));
+
+        assertArrayEquals(ring.tokens().toArray(), Arrays.asList(BigInteger.valueOf(0L),
+                                                                 BigInteger.valueOf(1L),
+                                                                 BigInteger.valueOf(100L),
+                                                                 BigInteger.valueOf(101L),
+                                                                 BigInteger.valueOf(200L),
+                                                                 BigInteger.valueOf(201L)).toArray());
+
+        assertArrayEquals(ring.tokens("DC1").toArray(), Arrays.asList(BigInteger.valueOf(0L),
+                                                                      BigInteger.valueOf(100L),
+                                                                      BigInteger.valueOf(200L)).toArray());
+
+        assertArrayEquals(ring.tokens("DC2").toArray(), Arrays.asList(BigInteger.valueOf(1L),
+                                                                      BigInteger.valueOf(101L),
+                                                                      BigInteger.valueOf(201L)).toArray());
+
+        Multimap<CassandraInstance, Range<BigInteger>> tokenRanges = ring.tokenRanges();
+        for (CassandraInstance instance : instances)
+        {
+            assertEquals(mergeRanges(tokenRanges.get(instance)),
+                         Range.closed(Partitioner.Murmur3Partitioner.minToken(),
+                                      Partitioner.Murmur3Partitioner.maxToken()));
+        }
+    }
+
+    @Test
+    public void testNetworkStrategyRF11()
+    {
+        List<CassandraInstance> instances = Arrays.asList(new CassandraInstance("0", "local0-i1", "DC1"),
+                                                          new CassandraInstance("100", "local0-i2", "DC1"),
+                                                          new CassandraInstance("200", "local0-i3", "DC1"),
+                                                          new CassandraInstance("1", "local1-i1", "DC2"),
+                                                          new CassandraInstance("101", "local1-i2", "DC2"),
+                                                          new CassandraInstance("201", "local1-i3", "DC2"));
+
+        CassandraRing ring = new CassandraRing(
+                Partitioner.Murmur3Partitioner,
+                "test",
+                new ReplicationFactor(ImmutableMap.of("class", "org.apache.cassandra.locator.NetworkTopologyStrategy",
+                                                      "DC1", "1",
+                                                      "DC2", "1")),
+                Arrays.asList(new CassandraInstance("0", "local0-i1", "DC1"),
+                              new CassandraInstance("100", "local0-i2", "DC1"),
+                              new CassandraInstance("200", "local0-i3", "DC1"),
+                              new CassandraInstance("1", "local1-i1", "DC2"),
+                              new CassandraInstance("101", "local1-i2", "DC2"),
+                              new CassandraInstance("201", "local1-i3", "DC2")));
+
+        assertArrayEquals(ring.tokens().toArray(), Arrays.asList(BigInteger.valueOf(0L),
+                                                                 BigInteger.valueOf(1L),
+                                                                 BigInteger.valueOf(100L),
+                                                                 BigInteger.valueOf(101L),
+                                                                 BigInteger.valueOf(200L),
+                                                                 BigInteger.valueOf(201L)).toArray());
+
+        assertArrayEquals(ring.tokens("DC1").toArray(), Arrays.asList(BigInteger.valueOf(0L),
+                                                                      BigInteger.valueOf(100L),
+                                                                      BigInteger.valueOf(200L)).toArray());
+
+        assertArrayEquals(ring.tokens("DC2").toArray(), Arrays.asList(BigInteger.valueOf(1L),
+                                                                      BigInteger.valueOf(101L),
+                                                                      BigInteger.valueOf(201L)).toArray());
+
+        Multimap<CassandraInstance, Range<BigInteger>> tokenRanges = ring.tokenRanges();
+
+        // token(0) => [201 - 0] => [201 - MAX], [MIN - 0]
+        validateRanges(tokenRanges.get(instances.get(0)),
+                       Arrays.asList(BigInteger.ZERO,
+                                     Partitioner.Murmur3Partitioner.minToken(),
+                                     Partitioner.Murmur3Partitioner.maxToken(),
+                                     BigInteger.valueOf(201L)),
+                       Arrays.asList(BigInteger.valueOf(200L),
+                                     BigInteger.valueOf(100L),
+                                     BigInteger.valueOf(1L)));
+
+        // token(100) => [1 - 100]
+        validateRanges(tokenRanges.get(instances.get(1)),
+                       Arrays.asList(BigInteger.valueOf(1L),
+                                     BigInteger.valueOf(100L),
+                                     BigInteger.valueOf(50L)),
+                       Arrays.asList(BigInteger.valueOf(101L),
+                                     BigInteger.valueOf(200L),
+                                     BigInteger.valueOf(0L),
+                                     Partitioner.Murmur3Partitioner.minToken(),
+                                     Partitioner.Murmur3Partitioner.maxToken()));
+
+        // token(200) => [101 - 200]
+        validateRanges(tokenRanges.get(instances.get(2)),
+                       Arrays.asList(BigInteger.valueOf(101L),
+                                     BigInteger.valueOf(150L),
+                                     BigInteger.valueOf(200L)),
+                       Arrays.asList(BigInteger.valueOf(100L),
+                                     BigInteger.valueOf(201L),
+                                     BigInteger.valueOf(1L),
+                                     Partitioner.Murmur3Partitioner.minToken(),
+                                     Partitioner.Murmur3Partitioner.maxToken()));
+
+        // token(1) => [202 - 1] => [202 - MAX], [MIN - 1]
+        validateRanges(tokenRanges.get(instances.get(3)),
+                       Arrays.asList(BigInteger.ONE,
+                                     Partitioner.Murmur3Partitioner.minToken(),
+                                     Partitioner.Murmur3Partitioner.maxToken(),
+                                     BigInteger.valueOf(202L)),
+                       Arrays.asList(BigInteger.valueOf(201L),
+                                     BigInteger.valueOf(101L),
+                                     BigInteger.valueOf(2L)));
+
+        // token(101) => [2 - 101]
+        validateRanges(tokenRanges.get(instances.get(4)),
+                       Arrays.asList(BigInteger.valueOf(2L),
+                                     BigInteger.valueOf(101L),
+                                     BigInteger.valueOf(50L)),
+                       Arrays.asList(BigInteger.valueOf(102L),
+                                     BigInteger.valueOf(201L),
+                                     BigInteger.valueOf(1L),
+                                     Partitioner.Murmur3Partitioner.minToken(),
+                                     Partitioner.Murmur3Partitioner.maxToken()));
+
+        // token(201) => [102 - 201]
+        validateRanges(tokenRanges.get(instances.get(5)),
+                       Arrays.asList(BigInteger.valueOf(102L),
+                                     BigInteger.valueOf(151L),
+                                     BigInteger.valueOf(201L)),
+                       Arrays.asList(BigInteger.valueOf(101L),
+                                     BigInteger.valueOf(202L),
+                                     BigInteger.valueOf(2L),
+                                     Partitioner.Murmur3Partitioner.minToken(),
+                                     Partitioner.Murmur3Partitioner.maxToken()));
+    }
+
+    @Test
+    public void testNetworkStrategyRF22()
+    {
+        List<CassandraInstance> instances = Arrays.asList(new CassandraInstance("0", "local0-i1", "DC1"),
+                                                          new CassandraInstance("100", "local0-i2", "DC1"),
+                                                          new CassandraInstance("200", "local0-i3", "DC1"),
+                                                          new CassandraInstance("1", "local1-i1", "DC2"),
+                                                          new CassandraInstance("101", "local1-i2", "DC2"),
+                                                          new CassandraInstance("201", "local1-i3", "DC2"));
+
+        CassandraRing ring = new CassandraRing(
+                Partitioner.Murmur3Partitioner,
+                "test",
+                new ReplicationFactor(ImmutableMap.of("class", "org.apache.cassandra.locator.NetworkTopologyStrategy",
+                                                      "DC1", "2",
+                                                      "DC2", "2")),
+                Arrays.asList(new CassandraInstance("0", "local0-i1", "DC1"),
+                              new CassandraInstance("100", "local0-i2", "DC1"),
+                              new CassandraInstance("200", "local0-i3", "DC1"),
+                              new CassandraInstance("1", "local1-i1", "DC2"),
+                              new CassandraInstance("101", "local1-i2", "DC2"),
+                              new CassandraInstance("201", "local1-i3", "DC2")));
+
+        assertArrayEquals(ring.tokens().toArray(), Arrays.asList(BigInteger.valueOf(0L),
+                                                                 BigInteger.valueOf(1L),
+                                                                 BigInteger.valueOf(100L),
+                                                                 BigInteger.valueOf(101L),
+                                                                 BigInteger.valueOf(200L),
+                                                                 BigInteger.valueOf(201L)).toArray());
+
+        assertArrayEquals(ring.tokens("DC1").toArray(), Arrays.asList(BigInteger.valueOf(0L),
+                                                                      BigInteger.valueOf(100L),
+                                                                      BigInteger.valueOf(200L)).toArray());
+
+        assertArrayEquals(ring.tokens("DC2").toArray(), Arrays.asList(BigInteger.valueOf(1L),
+                                                                      BigInteger.valueOf(101L),
+                                                                      BigInteger.valueOf(201L)).toArray());
+
+        Multimap<CassandraInstance, Range<BigInteger>> tokenRanges = ring.tokenRanges();
+
+        // token(0) => [101 - 0] => [101 - MAX] [MIN - 0]
+        validateRanges(tokenRanges.get(instances.get(0)),
+                       Arrays.asList(BigInteger.ZERO,
+                                     Partitioner.Murmur3Partitioner.minToken(),
+                                     Partitioner.Murmur3Partitioner.maxToken(),
+                                     BigInteger.valueOf(200L),
+                                     BigInteger.valueOf(101L)),
+                       Arrays.asList(BigInteger.valueOf(100L),
+                                     BigInteger.valueOf(1L)));
+
+        // token(100) => [201 - 100] => [201 - MAX] [MIN - 100]
+        validateRanges(tokenRanges.get(instances.get(1)),
+                       Arrays.asList(BigInteger.valueOf(0L),
+                                     Partitioner.Murmur3Partitioner.minToken(),
+                                     Partitioner.Murmur3Partitioner.maxToken(),
+                                     BigInteger.valueOf(100L),
+                                     BigInteger.valueOf(201L)),
+                       Arrays.asList(BigInteger.valueOf(101L),
+                                     BigInteger.valueOf(200L)));
+
+        // token(200) => [1 - 200]
+        validateRanges(tokenRanges.get(instances.get(2)),
+                       Arrays.asList(BigInteger.valueOf(1L),
+                                     BigInteger.valueOf(100L),
+                                     BigInteger.valueOf(200L)),
+                       Arrays.asList(BigInteger.valueOf(0L),
+                                     BigInteger.valueOf(201L),
+                                     Partitioner.Murmur3Partitioner.minToken(),
+                                     Partitioner.Murmur3Partitioner.maxToken()));
+
+        // token(1) => [102 - 1] => [102 - MAX] [MIN - 1]
+        validateRanges(tokenRanges.get(instances.get(3)),
+                       Arrays.asList(BigInteger.ONE,
+                                     Partitioner.Murmur3Partitioner.minToken(),
+                                     Partitioner.Murmur3Partitioner.maxToken(),
+                                     BigInteger.valueOf(200L),
+                                     BigInteger.valueOf(102L)),
+                       Arrays.asList(BigInteger.valueOf(101L),
+                                     BigInteger.valueOf(2L)));
+
+        // token(101) => [202 - 101] => [202 - MAX] [MIN - 101]
+        validateRanges(tokenRanges.get(instances.get(4)),
+                       Arrays.asList(BigInteger.valueOf(1L),
+                                     Partitioner.Murmur3Partitioner.minToken(),
+                                     Partitioner.Murmur3Partitioner.maxToken(),
+                                     BigInteger.valueOf(101L),
+                                     BigInteger.valueOf(202L)),
+                       Arrays.asList(BigInteger.valueOf(102L),
+                                     BigInteger.valueOf(201L)));
+
+        // token(201) => [2 - 201]
+        validateRanges(tokenRanges.get(instances.get(5)),
+                       Arrays.asList(BigInteger.valueOf(2L),
+                                     BigInteger.valueOf(101L),
+                                     BigInteger.valueOf(201L)),
+                       Arrays.asList(BigInteger.valueOf(1L),
+                                     BigInteger.valueOf(202L),
+                                     Partitioner.Murmur3Partitioner.minToken(),
+                                     Partitioner.Murmur3Partitioner.maxToken()));
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/partitioner/ConsistencyLevelTests.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/partitioner/ConsistencyLevelTests.java
new file mode 100644
index 0000000..30fb6c8
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/partitioner/ConsistencyLevelTests.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.cassandra.spark.data.partitioner;
+
+import com.google.common.collect.ImmutableMap;
+import org.junit.Test;
+
+import org.apache.cassandra.spark.data.ReplicationFactor;
+
+import static org.junit.Assert.assertEquals;
+
+public class ConsistencyLevelTests
+{
+    @Test
+    public void testSimpleStrategy()
+    {
+        assertEquals(1, ConsistencyLevel.ONE.blockFor(
+                new ReplicationFactor(ReplicationFactor.ReplicationStrategy.SimpleStrategy,
+                                      ImmutableMap.of("replication_factor", 3)), null));
+        assertEquals(1, ConsistencyLevel.ONE.blockFor(
+                new ReplicationFactor(ReplicationFactor.ReplicationStrategy.SimpleStrategy,
+                                      ImmutableMap.of("replication_factor", 1)), null));
+        assertEquals(2, ConsistencyLevel.TWO.blockFor(
+                new ReplicationFactor(ReplicationFactor.ReplicationStrategy.SimpleStrategy,
+                                      ImmutableMap.of("replication_factor", 3)), null));
+        assertEquals(3, ConsistencyLevel.THREE.blockFor(
+                new ReplicationFactor(ReplicationFactor.ReplicationStrategy.SimpleStrategy,
+                                      ImmutableMap.of("replication_factor", 3)), null));
+        assertEquals(1, ConsistencyLevel.LOCAL_ONE.blockFor(
+                new ReplicationFactor(ReplicationFactor.ReplicationStrategy.SimpleStrategy,
+                                      ImmutableMap.of("replication_factor", 3)), null));
+        assertEquals(2, ConsistencyLevel.LOCAL_QUORUM.blockFor(
+                new ReplicationFactor(ReplicationFactor.ReplicationStrategy.SimpleStrategy,
+                                      ImmutableMap.of("replication_factor", 3)), null));
+        assertEquals(3, ConsistencyLevel.LOCAL_QUORUM.blockFor(
+                new ReplicationFactor(ReplicationFactor.ReplicationStrategy.SimpleStrategy,
+                                      ImmutableMap.of("replication_factor", 5)), null));
+    }
+
+    @Test
+    public void testNetworkTopolgyStrategy()
+    {
+        assertEquals(1, ConsistencyLevel.ONE.blockFor(
+                new ReplicationFactor(ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy,
+                                      ImmutableMap.of("DC1", 3)), null));
+        assertEquals(1, ConsistencyLevel.ONE.blockFor(
+                new ReplicationFactor(ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy,
+                                      ImmutableMap.of("DC1", 1)), null));
+        assertEquals(2, ConsistencyLevel.TWO.blockFor(
+                new ReplicationFactor(ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy,
+                                      ImmutableMap.of("DC1", 3)), null));
+        assertEquals(3, ConsistencyLevel.THREE.blockFor(
+                new ReplicationFactor(ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy,
+                                      ImmutableMap.of("DC1", 3)), null));
+        assertEquals(1, ConsistencyLevel.LOCAL_ONE.blockFor(
+                new ReplicationFactor(ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy,
+                                      ImmutableMap.of("DC1", 3)), null));
+        assertEquals(2, ConsistencyLevel.LOCAL_QUORUM.blockFor(
+                new ReplicationFactor(ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy,
+                                      ImmutableMap.of("DC1", 3)), "DC1"));
+        assertEquals(3, ConsistencyLevel.LOCAL_QUORUM.blockFor(
+                new ReplicationFactor(ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy,
+                                      ImmutableMap.of("DC1", 5)), "DC1"));
+        assertEquals(2, ConsistencyLevel.LOCAL_QUORUM.blockFor(
+                new ReplicationFactor(ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy,
+                                      ImmutableMap.of("DC1", 3)), null));
+        assertEquals(3, ConsistencyLevel.LOCAL_QUORUM.blockFor(
+                new ReplicationFactor(ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy,
+                                      ImmutableMap.of("DC1", 5)), null));
+
+        assertEquals(2, ConsistencyLevel.LOCAL_QUORUM.blockFor(
+                new ReplicationFactor(ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy,
+                                      ImmutableMap.of("DC1", 3, "DC2", 5, "DC3", 4)), "DC1"));
+        assertEquals(3, ConsistencyLevel.LOCAL_QUORUM.blockFor(
+                new ReplicationFactor(ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy,
+                                      ImmutableMap.of("DC1", 3, "DC2", 5, "DC3", 4)), "DC2"));
+        assertEquals(3, ConsistencyLevel.LOCAL_QUORUM.blockFor(
+                new ReplicationFactor(ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy,
+                                      ImmutableMap.of("DC1", 3, "DC2", 5, "DC3", 4)), "DC3"));
+
+        assertEquals(5, ConsistencyLevel.EACH_QUORUM.blockFor(
+                new ReplicationFactor(ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy,
+                                      ImmutableMap.of("DC1", 3, "DC2", 5)), null));
+        assertEquals(8, ConsistencyLevel.EACH_QUORUM.blockFor(
+                new ReplicationFactor(ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy,
+                                      ImmutableMap.of("DC1", 3, "DC2", 5, "DC3", 4)), null));
+
+        assertEquals(5, ConsistencyLevel.ALL.blockFor(
+                new ReplicationFactor(ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy,
+                                      ImmutableMap.of("DC1", 5)), null));
+        assertEquals(10, ConsistencyLevel.ALL.blockFor(
+                new ReplicationFactor(ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy,
+                                      ImmutableMap.of("DC1", 5, "DC2", 5)), null));
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/partitioner/JDKSerializationTests.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/partitioner/JDKSerializationTests.java
new file mode 100644
index 0000000..dc866b5
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/partitioner/JDKSerializationTests.java
@@ -0,0 +1,312 @@
+/*
+ * 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.cassandra.spark.data.partitioner;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.math.BigInteger;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.stream.Stream;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Range;
+import org.apache.commons.lang.NotImplementedException;
+import org.apache.commons.lang3.RandomStringUtils;
+import org.junit.Test;
+
+import org.apache.cassandra.bridge.CassandraBridge;
+import org.apache.cassandra.bridge.CassandraBridgeFactory;
+import org.apache.cassandra.bridge.CassandraVersion;
+import org.apache.cassandra.clients.SslConfig;
+import org.apache.cassandra.spark.TestUtils;
+import org.apache.cassandra.spark.cdc.CommitLog;
+import org.apache.cassandra.spark.cdc.CommitLogProvider;
+import org.apache.cassandra.spark.cdc.TableIdLookup;
+import org.apache.cassandra.spark.data.CqlField;
+import org.apache.cassandra.spark.data.CqlTable;
+import org.apache.cassandra.spark.data.DataLayer;
+import org.apache.cassandra.spark.data.PartitionedDataLayer;
+import org.apache.cassandra.spark.data.ReplicationFactor;
+import org.apache.cassandra.spark.data.SSTable;
+import org.apache.cassandra.spark.data.VersionRunner;
+import org.apache.cassandra.spark.utils.test.TestSchema;
+import org.jetbrains.annotations.NotNull;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.quicktheories.QuickTheory.qt;
+import static org.quicktheories.generators.SourceDSL.arbitrary;
+
+public class JDKSerializationTests extends VersionRunner
+{
+    public JDKSerializationTests(CassandraVersion version)
+    {
+        super(version);
+    }
+
+    @Test
+    public void testCassandraRing()
+    {
+        qt().forAll(TestUtils.partitioners(), arbitrary().pick(Arrays.asList(1, 3, 6, 12, 128)))
+            .checkAssert(((partitioner, numInstances) -> {
+                CassandraRing ring;
+                if (numInstances > 4)
+                {
+                    ring = TestUtils.createRing(partitioner, ImmutableMap.of("DC1", numInstances / 2, "DC2", numInstances / 2));
+                }
+                else
+                {
+                    ring = TestUtils.createRing(partitioner, numInstances);
+                }
+                byte[] bytes = bridge.javaSerialize(ring);
+                CassandraRing deserialized = bridge.javaDeserialize(bytes, CassandraRing.class);
+                assertNotNull(deserialized);
+                assertNotNull(deserialized.rangeMap());
+                assertNotNull(deserialized.tokenRanges());
+                assertEquals(ring, deserialized);
+            }));
+    }
+
+    @Test
+    public void testTokenPartitioner()
+    {
+        qt().forAll(TestUtils.partitioners(),
+                    arbitrary().pick(Arrays.asList(1, 3, 6, 12, 128)),
+                    arbitrary().pick(Arrays.asList(1, 4, 8, 16, 32, 1024)))
+            .checkAssert(((partitioner, numInstances, numCores) -> {
+                CassandraRing ring = TestUtils.createRing(partitioner, numInstances);
+                TokenPartitioner tokenPartitioner = new TokenPartitioner(ring, 4, numCores);
+                byte[] bytes = bridge.javaSerialize(tokenPartitioner);
+                TokenPartitioner deserialized = bridge.javaDeserialize(bytes, TokenPartitioner.class);
+                assertEquals(tokenPartitioner.ring(), deserialized.ring());
+                assertEquals(tokenPartitioner.numPartitions(), deserialized.numPartitions());
+                assertEquals(tokenPartitioner.subRanges(), deserialized.subRanges());
+                assertEquals(tokenPartitioner.partitionMap(), deserialized.partitionMap());
+                assertEquals(tokenPartitioner.reversePartitionMap(), deserialized.reversePartitionMap());
+                for (int partition = 0; partition < tokenPartitioner.numPartitions(); partition++)
+                {
+                    assertEquals(tokenPartitioner.getTokenRange(partition), deserialized.getTokenRange(partition));
+                }
+            }));
+    }
+
+    @Test
+    public void testPartitionedDataLayer()
+    {
+        CassandraRing ring = TestUtils.createRing(Partitioner.Murmur3Partitioner, 1024);
+        TestSchema schema = TestSchema.basic(bridge);
+        CqlTable cqlTable = new CqlTable(schema.keyspace, schema.table, schema.createStatement, ring.replicationFactor(), Collections.emptyList());
+        DataLayer partitionedDataLayer = new TestPartitionedDataLayer(bridge, 4, 16, null, ring, cqlTable);
+        byte[] bytes = bridge.javaSerialize(partitionedDataLayer);
+        TestPartitionedDataLayer deserialized = bridge.javaDeserialize(bytes, TestPartitionedDataLayer.class);
+        assertNotNull(deserialized);
+        assertNotNull(deserialized.ring());
+        assertNotNull(deserialized.partitioner());
+        assertNotNull(deserialized.tokenPartitioner());
+        assertEquals(Partitioner.Murmur3Partitioner, deserialized.partitioner());
+    }
+
+    @Test
+    public void testCqlFieldSet()
+    {
+        CqlField.CqlSet setType = bridge.set(bridge.text());
+        CqlField field = new CqlField(true, false, false, RandomStringUtils.randomAlphanumeric(5, 20), setType, 10);
+        byte[] bytes = bridge.javaSerialize(field);
+        CqlField deserialized = bridge.javaDeserialize(bytes, CqlField.class);
+        assertEquals(field, deserialized);
+        assertEquals(field.name(), deserialized.name());
+        assertEquals(field.type(), deserialized.type());
+        assertEquals(field.position(), deserialized.position());
+        assertEquals(field.isPartitionKey(), deserialized.isPartitionKey());
+        assertEquals(field.isClusteringColumn(), deserialized.isClusteringColumn());
+    }
+
+    @Test
+    public void testCqlUdt()
+    {
+        CqlField.CqlUdt udt1 = bridge
+                               .udt("udt_keyspace", "udt_table")
+                               .withField("c", bridge.text())
+                               .withField("b", bridge.timestamp())
+                               .withField("a", bridge.bigint())
+                               .build();
+        CqlField.CqlUdt udt2 = bridge
+                               .udt("udt_keyspace", "udt_table")
+                               .withField("a", bridge.bigint())
+                               .withField("b", bridge.timestamp())
+                               .withField("c", bridge.text())
+                               .build();
+        assertNotEquals(udt2, udt1);
+        byte[] bytes = bridge.javaSerialize(udt1);
+        CqlField.CqlUdt deserialized = bridge.javaDeserialize(bytes, CqlField.CqlUdt.class);
+        assertEquals(udt1, deserialized);
+        assertNotEquals(udt2, deserialized);
+        for (int field = 0; field < deserialized.fields().size(); field++)
+        {
+            assertEquals(udt1.field(field), deserialized.field(field));
+        }
+    }
+
+    public static class TestPartitionedDataLayer extends PartitionedDataLayer
+    {
+        private CassandraBridge bridge;
+        private CassandraRing ring;
+        private CqlTable cqlTable;
+        private TokenPartitioner tokenPartitioner;
+        private final String jobId;
+
+        public TestPartitionedDataLayer(CassandraBridge bridge,
+                                        int defaultParallelism,
+                                        int numCores,
+                                        String dc,
+                                        CassandraRing ring,
+                                        CqlTable cqlTable)
+        {
+            super(ConsistencyLevel.LOCAL_QUORUM, dc);
+            this.bridge = bridge;
+            this.ring = ring;
+            this.cqlTable = cqlTable;
+            this.tokenPartitioner = new TokenPartitioner(ring, defaultParallelism, numCores);
+            this.jobId = UUID.randomUUID().toString();
+        }
+
+        public CompletableFuture<Stream<SSTable>> listInstance(int partitionId,
+                                                               @NotNull Range<BigInteger> range,
+                                                               @NotNull CassandraInstance instance)
+        {
+            return CompletableFuture.completedFuture(Stream.of());
+        }
+
+        @Override
+        public CassandraBridge bridge()
+        {
+            return bridge;
+        }
+
+        @Override
+        public CassandraRing ring()
+        {
+            return ring;
+        }
+
+        public TokenPartitioner tokenPartitioner()
+        {
+            return tokenPartitioner;
+        }
+
+        @Override
+        public CompletableFuture<List<CommitLog>> listCommitLogs(CassandraInstance instance)
+        {
+            throw new NotImplementedException("Test listCommitLogs not implemented yet");
+        }
+
+        protected ExecutorService executorService()
+        {
+            return SingleReplicaTests.EXECUTOR;
+        }
+
+        public String jobId()
+        {
+            return jobId;
+        }
+
+        public CassandraVersion version()
+        {
+            return CassandraVersion.FOURZERO;
+        }
+
+        public CqlTable cqlTable()
+        {
+            return cqlTable;
+        }
+
+        public CommitLogProvider commitLogs(int partitionId)
+        {
+            throw new NotImplementedException("Test CommitLogProvider not implemented yet");
+        }
+
+        @Override
+        public ReplicationFactor replicationFactor(String keyspace)
+        {
+            return ring.replicationFactor();
+        }
+
+        @Override
+        public TableIdLookup tableIdLookup()
+        {
+            throw new NotImplementedException("Test TableIdLookup not implemented yet");
+        }
+
+        private void writeObject(ObjectOutputStream out) throws IOException
+        {
+            // Falling back to JDK serialization
+            out.writeObject(version());
+            out.writeObject(consistencyLevel);
+            out.writeObject(datacenter);
+            out.writeObject(ring);
+            bridge.javaSerialize(out, cqlTable);  // Delegate (de-)serialization of version-specific objects to the Cassandra Bridge
+            out.writeObject(tokenPartitioner);
+        }
+
+        private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException
+        {
+            // Falling back to JDK deserialization
+            bridge = CassandraBridgeFactory.get((CassandraVersion) in.readObject());
+            consistencyLevel = (ConsistencyLevel) in.readObject();
+            datacenter = (String) in.readObject();
+            ring = (CassandraRing) in.readObject();
+            cqlTable = bridge.javaDeserialize(in, CqlTable.class);  // Delegate (de-)serialization of version-specific objects to the Cassandra Bridge
+            tokenPartitioner = (TokenPartitioner) in.readObject();
+        }
+    }
+
+    @Test
+    public void testSecretsConfig()
+    {
+        SslConfig config = new SslConfig.Builder<>()
+                           .keyStorePath("keyStorePath")
+                           .base64EncodedKeyStore("encodedKeyStore")
+                           .keyStorePassword("keyStorePassword")
+                           .keyStoreType("keyStoreType")
+                           .trustStorePath("trustStorePath")
+                           .base64EncodedTrustStore("encodedTrustStore")
+                           .trustStorePassword("trustStorePassword")
+                           .trustStoreType("trustStoreType")
+                           .build();
+        byte[] bytes = bridge.javaSerialize(config);
+        SslConfig deserialized = bridge.javaDeserialize(bytes, SslConfig.class);
+
+        assertEquals(config.keyStorePath(), deserialized.keyStorePath());
+        assertEquals(config.base64EncodedKeyStore(), deserialized.base64EncodedKeyStore());
+        assertEquals(config.keyStorePassword(), deserialized.keyStorePassword());
+        assertEquals(config.keyStoreType(), deserialized.keyStoreType());
+        assertEquals(config.trustStorePath(), deserialized.trustStorePath());
+        assertEquals(config.base64EncodedTrustStore(), deserialized.base64EncodedTrustStore());
+        assertEquals(config.trustStorePassword(), deserialized.trustStorePassword());
+        assertEquals(config.trustStoreType(), deserialized.trustStoreType());
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/partitioner/MultipleReplicasTests.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/partitioner/MultipleReplicasTests.java
new file mode 100644
index 0000000..78d8615
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/partitioner/MultipleReplicasTests.java
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.cassandra.spark.data.partitioner;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+
+import com.google.common.collect.Range;
+import org.junit.Test;
+
+import org.apache.cassandra.spark.TestUtils;
+import org.apache.cassandra.spark.data.PartitionedDataLayer;
+import org.apache.cassandra.spark.data.SSTable;
+import org.apache.cassandra.spark.reader.SparkSSTableReader;
+import org.apache.cassandra.spark.stats.Stats;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Matchers.eq;
+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.quicktheories.QuickTheory.qt;
+
+public class MultipleReplicasTests
+{
+    private static final int[] NUM_SSTABLES = new int[]{3, 5, 7, 11, 13, 17, 19, 23};
+
+    @Test
+    public void testRF1AllUp()
+    {
+        runTest(3, 1, 0, 0);
+    }
+
+    @Test
+    public void testRF1BackupsDown()
+    {
+        runTest(3, 1, 0, 2);
+    }
+
+    @Test
+    public void testRF1SomeDown()
+    {
+        runTest(3, 1, 1, 1);
+    }
+
+    @Test
+    public void testRF3QuorumAllUp()
+    {
+        runTest(3, 2, 0, 0);
+    }
+
+    @Test
+    public void testRF3QuorumBackupInstanceDown()
+    {
+        runTest(3, 2, 0, 1);
+    }
+
+    @Test
+    public void testRF3QuorumPrimaryInstanceDown()
+    {
+        runTest(3, 2, 1, 0);
+    }
+
+    @Test
+    public void testRF5QuorumTwoPrimaryInstanceDown()
+    {
+        runTest(5, 3, 2, 0);
+    }
+
+    @Test(expected = AssertionError.class)
+    public void testRF1NotEnoughReplicas()
+    {
+        runTest(1, 1, 1, 0);
+    }
+
+    @Test(expected = AssertionError.class)
+    public void testRF3QuorumNotEnoughReplicas()
+    {
+        runTest(3, 2, 1, 1);
+    }
+
+    @Test(expected = AssertionError.class)
+    public void testRFAllNotEnoughReplicas()
+    {
+        runTest(3, 3, 1, 0);
+    }
+
+    private static void runTest(int numInstances, int rfFactor, int numDownPrimaryInstances, int numDownBackupInstances)
+    {
+        qt().forAll(TestUtils.partitioners()).checkAssert(partitioner -> {
+            // Mock CassandraRing/Instances and DataLayer
+            CassandraRing ring = TestUtils.createRing(partitioner, numInstances);
+            List<CassandraInstance> instances = new ArrayList<>(ring.instances());
+            PartitionedDataLayer dataLayer = mock(PartitionedDataLayer.class);
+            Range<BigInteger> range = Range.closed(partitioner.minToken(), partitioner.maxToken());
+            Set<SingleReplica> primaryReplicas = new HashSet<>(rfFactor);
+            Set<SingleReplica> backupReplicas = new HashSet<>(numInstances - rfFactor);
+            int expectedSSTables = 0;
+            int upInstances = 0;
+            List<CassandraInstance> requestedInstances = new ArrayList<>();
+
+            // Mock some primary and backup replicas with a different number of SSTables and some UP some DOWN
+            for (int position = 0; position < rfFactor; position++)
+            {
+                boolean isDown = position < numDownPrimaryInstances;
+                int numSSTables = NUM_SSTABLES[position];
+                requestedInstances.add(instances.get(position));
+                if (!isDown)
+                {
+                    upInstances++;
+                    expectedSSTables += numSSTables;
+                }
+                primaryReplicas.add(mockReplica(instances.get(position), dataLayer, range, numSSTables, isDown));
+            }
+            for (int position = rfFactor; position < numInstances; position++)
+            {
+                boolean isDown = (position - rfFactor) < numDownBackupInstances;
+                int numSSTables = NUM_SSTABLES[position];
+                SingleReplica replica = mockReplica(instances.get(position), dataLayer, range, numSSTables, isDown);
+                if (!isDown && upInstances < rfFactor)
+                {
+                    upInstances++;
+                    expectedSSTables += numSSTables;
+                    requestedInstances.add(instances.get(position));
+                }
+                backupReplicas.add(replica);
+            }
+
+            // Open replicas and verify correct number of SSTables opened should only throw NotEnoughReplicasException
+            // if insufficient primary or backup replicas available to meet consistency level
+            MultipleReplicas replicas = new MultipleReplicas(primaryReplicas, backupReplicas, Stats.DoNothingStats.INSTANCE);
+            Set<TestSSTableReader> readers = replicas.openAll((ssTable, isRepairPrimary) -> new TestSSTableReader(ssTable));
+            assertEquals(expectedSSTables, readers.size());
+
+            // Verify list instance attempted on all primary instances
+            // and any backup instances that needed to be called to meet consistency
+            for (CassandraInstance instance : requestedInstances)
+            {
+                verify(dataLayer, times(1)).listInstance(eq(0), eq(range), eq(instance));
+            }
+        });
+    }
+
+    private static SingleReplica mockReplica(CassandraInstance instance,
+                                             PartitionedDataLayer dataLayer,
+                                             Range<BigInteger> range,
+                                             int numSSTables,
+                                             boolean shouldFail)
+    {
+        when(dataLayer.listInstance(eq(0), eq(range), eq(instance))).thenAnswer(invocation -> {
+            if (shouldFail)
+            {
+                CompletableFuture<Stream<SSTable>> exceptionally = new CompletableFuture<>();
+                exceptionally.completeExceptionally(new RuntimeException("Something went wrong"));
+                return exceptionally;
+            }
+            return CompletableFuture.completedFuture(IntStream.range(0, numSSTables)
+                                                              .mapToObj(ssTable -> SingleReplicaTests.mockSSTable()));
+        });
+        return new SingleReplica(instance, dataLayer, range, 0, SingleReplicaTests.EXECUTOR, true);
+    }
+
+    public static class TestSSTableReader implements SparkSSTableReader
+    {
+        public TestSSTableReader(SSTable ssTable)
+        {
+        }
+
+        public BigInteger firstToken()
+        {
+            return BigInteger.valueOf(-4099276460824344804L);
+        }
+
+        public BigInteger lastToken()
+        {
+            return BigInteger.valueOf(2049638230412172401L);
+        }
+
+        public boolean ignore()
+        {
+            return false;
+        }
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/partitioner/SingleReplicaTests.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/partitioner/SingleReplicaTests.java
new file mode 100644
index 0000000..4bca3a4
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/partitioner/SingleReplicaTests.java
@@ -0,0 +1,238 @@
+/*
+ * 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.cassandra.spark.data.partitioner;
+
+import java.io.IOException;
+import java.math.BigInteger;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.stream.Stream;
+
+import com.google.common.collect.Range;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.junit.Test;
+
+import org.apache.cassandra.spark.data.FileType;
+import org.apache.cassandra.spark.data.IncompleteSSTableException;
+import org.apache.cassandra.spark.data.PartitionedDataLayer;
+import org.apache.cassandra.spark.data.SSTable;
+import org.apache.cassandra.spark.data.SSTablesSupplier;
+import org.apache.cassandra.spark.reader.SparkSSTableReader;
+import org.apache.cassandra.spark.reader.common.SSTableStreamException;
+import org.jetbrains.annotations.Nullable;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.doCallRealMethod;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class SingleReplicaTests
+{
+    public static final ExecutorService EXECUTOR =
+            Executors.newSingleThreadExecutor(new ThreadFactoryBuilder().setNameFormat("replicas-tests-%d")
+                                                                        .setDaemon(true)
+                                                                        .build());
+
+    @Test
+    public void testOpenSSTables() throws ExecutionException, InterruptedException, IOException
+    {
+        runTest(false);  // Missing no files
+    }
+
+    @Test
+    public void testMissingNonEssentialFiles() throws ExecutionException, InterruptedException, IOException
+    {
+        runTest(false, FileType.FILTER);  // Missing non-essential SSTable file component
+    }
+
+    @Test
+    public void testMissingOnlySummaryFile() throws ExecutionException, InterruptedException, IOException
+    {
+        // Summary.db can be missing if we can use Index.db
+        runTest(false, FileType.SUMMARY);
+    }
+
+    @Test
+    public void testMissingOnlyIndexFile() throws ExecutionException, InterruptedException, IOException
+    {
+        // Index.db can be missing if we can use Summary.db
+        runTest(false, FileType.INDEX);
+    }
+
+    @Test(expected = IOException.class)
+    public void testMissingDataFile() throws ExecutionException, InterruptedException, IOException
+    {
+        runTest(true, FileType.DATA);
+    }
+
+    @Test(expected = IOException.class)
+    public void testMissingStatisticsFile() throws ExecutionException, InterruptedException, IOException
+    {
+        runTest(true, FileType.STATISTICS);
+    }
+
+    @Test(expected = IOException.class)
+    public void testMissingSummaryPrimaryIndex() throws ExecutionException, InterruptedException, IOException
+    {
+        runTest(true, FileType.SUMMARY, FileType.INDEX);
+    }
+
+    @Test(expected = IOException.class)
+    public void testFailOpenReader() throws ExecutionException, InterruptedException, IOException
+    {
+        runTest(true,
+                (ssTable, isRepairPrimary) -> {
+                    throw new IOException("Couldn't open Summary.db file");
+                },
+                Range.closed(BigInteger.valueOf(-9223372036854775808L), BigInteger.valueOf(8710962479251732707L)));
+    }
+
+    @Test
+    public void testFilterOverlap() throws ExecutionException, InterruptedException, IOException
+    {
+        // Should not filter out SSTables overlapping with token range
+        runTest(false,
+                (ssTable, isRepairPrimary) -> new Reader(ssTable, BigInteger.valueOf(50), BigInteger.valueOf(150L)),
+                Range.closed(BigInteger.valueOf(0L), BigInteger.valueOf(100L)));
+    }
+
+    @Test
+    public void testFilterInnerlap() throws ExecutionException, InterruptedException, IOException
+    {
+        // Should not filter out SSTables overlapping with token range
+        runTest(false,
+                (ssTable, isRepairPrimary) -> new Reader(ssTable, BigInteger.valueOf(25), BigInteger.valueOf(75L)),
+                Range.closed(BigInteger.valueOf(0L), BigInteger.valueOf(100L)));
+    }
+
+    @Test
+    public void testFilterBoundary() throws ExecutionException, InterruptedException, IOException
+    {
+        // Should not filter out SSTables overlapping with token range
+        runTest(false,
+                (ssTable, isRepairPrimary) -> new Reader(ssTable, BigInteger.valueOf(100L), BigInteger.valueOf(102L)),
+                Range.closed(BigInteger.valueOf(0L), BigInteger.valueOf(100L)));
+    }
+
+    private static void runTest(
+            boolean shouldThrowIOException,
+            FileType... missingFileTypes) throws ExecutionException, InterruptedException, IOException
+    {
+        runTest(shouldThrowIOException,
+                (ssTable, isRepairPrimary) -> new Reader(ssTable),
+                Range.closed(BigInteger.valueOf(-9223372036854775808L), BigInteger.valueOf(8710962479251732707L)),
+                missingFileTypes);
+    }
+
+    private static void runTest(
+            boolean shouldThrowIOException,
+            SSTablesSupplier.ReaderOpener<Reader> readerOpener,
+            Range<BigInteger> range,
+            FileType... missingFileTypes) throws InterruptedException, IOException, ExecutionException
+    {
+        PartitionedDataLayer dataLayer = mock(PartitionedDataLayer.class);
+        CassandraInstance instance = new CassandraInstance("-9223372036854775808", "local1-i1", "DC1");
+
+        SSTable ssTable1 = mockSSTable();
+        SSTable ssTable2 = mockSSTable();
+        SSTable ssTable3 = mockSSTable();
+        for (FileType fileType : missingFileTypes)
+        {
+            // verify() should throw IncompleteSSTableException when missing Statistic.db file
+            when(ssTable3.isMissing(eq(fileType))).thenReturn(true);
+        }
+
+        Stream<SSTable> sstables = Stream.of(ssTable1, ssTable2, ssTable3);
+        when(dataLayer.listInstance(eq(0), eq(range), eq(instance)))
+                .thenReturn(CompletableFuture.completedFuture(sstables));
+
+        SingleReplica replica = new SingleReplica(instance, dataLayer, range, 0, EXECUTOR, true);
+        Set<Reader> readers;
+        try
+        {
+            readers = replica.openReplicaAsync(readerOpener).get();
+        }
+        catch (ExecutionException exception)
+        {
+            // Extract IOException and rethrow if wrapped in SSTableStreamException
+            IOException io = SSTableStreamException.getIOException(exception);
+            if (io != null)
+            {
+                throw io;
+            }
+            throw exception;
+        }
+        if (shouldThrowIOException)
+        {
+            fail("Should throw IOException because an SSTable is corrupt");
+        }
+        assertEquals(3, readers.size());
+    }
+
+    static SSTable mockSSTable() throws IncompleteSSTableException
+    {
+        SSTable ssTable = mock(SSTable.class);
+        when(ssTable.isMissing(any(FileType.class))).thenReturn(false);
+        doCallRealMethod().when(ssTable).verify();
+        return ssTable;
+    }
+
+    public static class Reader implements SparkSSTableReader
+    {
+        BigInteger firstToken;
+        BigInteger lastToken;
+        SSTable ssTable;
+
+        Reader(SSTable ssTable)
+        {
+            this(ssTable, BigInteger.valueOf(0L), BigInteger.valueOf(1L));
+        }
+
+        Reader(SSTable ssTable, @Nullable BigInteger firstToken, @Nullable BigInteger lastToken)
+        {
+            this.ssTable = ssTable;
+            this.firstToken = firstToken;
+            this.lastToken = lastToken;
+        }
+
+        @Override
+        public BigInteger firstToken()
+        {
+            return firstToken;
+        }
+
+        @Override
+        public BigInteger lastToken()
+        {
+            return lastToken;
+        }
+
+        public boolean ignore()
+        {
+            return false;
+        }
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/partitioner/TokenPartitionerTests.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/partitioner/TokenPartitionerTests.java
new file mode 100644
index 0000000..6eae4ce
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/data/partitioner/TokenPartitionerTests.java
@@ -0,0 +1,83 @@
+/*
+ * 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.cassandra.spark.data.partitioner;
+
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import com.google.common.collect.Range;
+import org.junit.Test;
+
+import org.apache.cassandra.spark.TestUtils;
+import org.apache.cassandra.spark.utils.RandomUtils;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.quicktheories.QuickTheory.qt;
+import static org.quicktheories.generators.SourceDSL.arbitrary;
+
+public class TokenPartitionerTests
+{
+    private static final int NUM_TOKEN_TESTS = 100;
+
+    @Test
+    public void testTokenPartitioner()
+    {
+        qt().forAll(TestUtils.partitioners(),
+                    arbitrary().pick(Arrays.asList(1, 3, 6, 12, 104, 208, 416)),
+                    arbitrary().pick(Arrays.asList(1, 2, 4, 16, 128, 1024)))
+            .checkAssert(this::runTest);
+    }
+
+    private void runTest(Partitioner partitioner, int numInstances, int numCores)
+    {
+        TokenPartitioner tokenPartitioner = new TokenPartitioner(TestUtils.createRing(partitioner, numInstances), 1, numCores);
+        assertTrue(tokenPartitioner.numPartitions() > 1);
+
+        // Generate some random tokens and verify they only exist in a single token partition
+        Map<BigInteger, Integer> tokens = IntStream.range(0, NUM_TOKEN_TESTS)
+                                                         .mapToObj(token -> RandomUtils.randomBigInteger(partitioner))
+                                                         .collect(Collectors.toMap(Function.identity(), token -> 0));
+
+        for (int partition = 0; partition < tokenPartitioner.numPartitions(); partition++)
+        {
+            Range<BigInteger> range = tokenPartitioner.getTokenRange(partition);
+            for (BigInteger token : tokens.keySet())
+            {
+                if (range.contains(token))
+                {
+                    tokens.put(token, tokens.get(token) + 1);
+                    assertTrue(tokenPartitioner.isInPartition(token, ByteBuffer.wrap("not important".getBytes()), partition));
+                }
+            }
+        }
+
+        for (Map.Entry<BigInteger, Integer> entry : tokens.entrySet())
+        {
+            assertFalse("Token not found in any token partitions: " + entry.getKey(), entry.getValue() < 1);
+            assertFalse("Token exists in more than one token partition: " + entry.getKey(), entry.getValue() > 1);
+        }
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/reader/CassandraBridgeTests.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/reader/CassandraBridgeTests.java
new file mode 100644
index 0000000..9a494c8
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/reader/CassandraBridgeTests.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.cassandra.spark.reader;
+
+import org.junit.Test;
+
+import org.apache.cassandra.spark.TestUtils;
+import org.apache.spark.sql.types.DataTypes;
+
+import static org.junit.Assert.assertEquals;
+import static org.quicktheories.QuickTheory.qt;
+
+public class CassandraBridgeTests
+{
+    @Test
+    public void testSparkDataTypes()
+    {
+        qt().forAll(TestUtils.bridges())
+            .checkAssert(bridge -> {
+                assertEquals(DataTypes.StringType, bridge.timeuuid().sparkSqlType());
+                assertEquals(DataTypes.StringType, bridge.uuid().sparkSqlType());
+                assertEquals(DataTypes.StringType, bridge.ascii().sparkSqlType());
+                assertEquals(DataTypes.StringType, bridge.varchar().sparkSqlType());
+                assertEquals(DataTypes.StringType, bridge.text().sparkSqlType());
+                assertEquals(DataTypes.BinaryType, bridge.inet().sparkSqlType());
+                assertEquals(DataTypes.BinaryType, bridge.blob().sparkSqlType());
+                assertEquals(DataTypes.IntegerType, bridge.aInt().sparkSqlType());
+                assertEquals(DataTypes.DateType, bridge.date().sparkSqlType());
+                assertEquals(DataTypes.LongType, bridge.bigint().sparkSqlType());
+                assertEquals(DataTypes.LongType, bridge.time().sparkSqlType());
+                assertEquals(DataTypes.BooleanType, bridge.bool().sparkSqlType());
+                assertEquals(DataTypes.FloatType, bridge.aFloat().sparkSqlType());
+                assertEquals(DataTypes.DoubleType, bridge.aDouble().sparkSqlType());
+                assertEquals(DataTypes.TimestampType, bridge.timestamp().sparkSqlType());
+                assertEquals(DataTypes.NullType, bridge.empty().sparkSqlType());
+                assertEquals(DataTypes.ShortType, bridge.smallint().sparkSqlType());
+                assertEquals(DataTypes.ByteType, bridge.tinyint().sparkSqlType());
+            });
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/reader/DataTypeSerializationTests.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/reader/DataTypeSerializationTests.java
new file mode 100644
index 0000000..ccd7414
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/reader/DataTypeSerializationTests.java
@@ -0,0 +1,569 @@
+/*
+ * 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.cassandra.spark.reader;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import org.junit.Test;
+
+import org.apache.cassandra.bridge.BigNumberConfig;
+import org.apache.cassandra.spark.TestUtils;
+import org.apache.cassandra.spark.data.CqlField;
+import org.apache.cassandra.spark.utils.RandomUtils;
+import org.apache.spark.sql.catalyst.util.ArrayBasedMapData;
+import org.apache.spark.sql.catalyst.util.ArrayData;
+import org.apache.spark.sql.types.Decimal;
+import org.apache.spark.unsafe.types.UTF8String;
+
+import static org.apache.cassandra.spark.TestUtils.runTest;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotSame;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.quicktheories.QuickTheory.qt;
+import static org.quicktheories.generators.SourceDSL.bigDecimals;
+import static org.quicktheories.generators.SourceDSL.bigIntegers;
+import static org.quicktheories.generators.SourceDSL.dates;
+import static org.quicktheories.generators.SourceDSL.doubles;
+import static org.quicktheories.generators.SourceDSL.floats;
+import static org.quicktheories.generators.SourceDSL.integers;
+import static org.quicktheories.generators.SourceDSL.longs;
+import static org.quicktheories.generators.SourceDSL.strings;
+
+public class DataTypeSerializationTests
+{
+    private static final int MAX_TESTS = 1000;
+
+    @Test
+    public void testVarInt()
+    {
+        qt().forAll(TestUtils.bridges()).checkAssert(bridge -> {
+            assertTrue(bridge.varint().deserialize(
+                    bridge.varint().serialize(BigInteger.valueOf(500L))) instanceof Decimal);
+            assertEquals(Decimal.apply(500), bridge.varint().deserialize(
+                    bridge.varint().serialize(BigInteger.valueOf(500L))));
+            assertNotSame(Decimal.apply(501), bridge.varint().deserialize(
+                    bridge.varint().serialize(BigInteger.valueOf(500L))));
+            assertEquals(Decimal.apply(-1), bridge.varint().deserialize(
+                    bridge.varint().serialize(BigInteger.valueOf(-1L))));
+            assertEquals(Decimal.apply(Long.MAX_VALUE), bridge.varint().deserialize(
+                    bridge.varint().serialize(BigInteger.valueOf(Long.MAX_VALUE))));
+            assertEquals(Decimal.apply(Long.MIN_VALUE), bridge.varint().deserialize(
+                    bridge.varint().serialize(BigInteger.valueOf(Long.MIN_VALUE))));
+            assertEquals(Decimal.apply(Integer.MAX_VALUE), bridge.varint().deserialize(
+                    bridge.varint().serialize(BigInteger.valueOf(Integer.MAX_VALUE))));
+            assertEquals(Decimal.apply(Integer.MIN_VALUE), bridge.varint().deserialize(
+                    bridge.varint().serialize(BigInteger.valueOf(Integer.MIN_VALUE))));
+            BigInteger veryLargeValue = BigInteger.valueOf(Integer.MAX_VALUE).multiply(BigInteger.valueOf(5));
+            assertEquals(Decimal.apply(veryLargeValue), bridge.varint().deserialize(
+                    bridge.varint().serialize(veryLargeValue)));
+            qt().withExamples(MAX_TESTS)
+                .forAll(bigIntegers().ofBytes(128))
+                .checkAssert(integer -> assertEquals(Decimal.apply(integer), bridge.varint().deserialize(
+                        bridge.varint().serialize(integer))));
+        });
+    }
+
+    @Test
+    public void testInt()
+    {
+        qt().forAll(TestUtils.bridges()).checkAssert(bridge -> {
+            assertTrue(bridge.aInt().deserialize(bridge.aInt().serialize(5)) instanceof Integer);
+            assertEquals(999, bridge.aInt().deserialize(ByteBuffer.allocate(4).putInt(0, 999)));
+            qt().forAll(integers().all())
+                .checkAssert(integer -> assertEquals(integer, bridge.aInt().deserialize(
+                        bridge.aInt().serialize(integer))));
+        });
+    }
+
+    @Test
+    public void testBoolean()
+    {
+        qt().forAll(TestUtils.bridges()).checkAssert(bridge -> {
+            assertTrue(bridge.bool().deserialize(bridge.bool().serialize(true)) instanceof Boolean);
+            assertTrue((Boolean) bridge.bool().deserialize(bridge.bool().serialize(true)));
+            assertFalse((Boolean) bridge.bool().deserialize(bridge.bool().serialize(false)));
+        });
+    }
+
+    @Test
+    public void testTimeUUID()
+    {
+        qt().forAll(TestUtils.bridges()).checkAssert(bridge -> {
+            assertTrue(bridge.timeuuid().deserialize(
+                    bridge.timeuuid().serialize(RandomUtils.getRandomTimeUUIDForTesting())) instanceof UTF8String);
+            for (int test = 0; test < MAX_TESTS; test++)
+            {
+                UUID expected = RandomUtils.getRandomTimeUUIDForTesting();
+                assertEquals(expected.toString(), bridge.timeuuid().deserialize(
+                        bridge.timeuuid().serialize(expected)).toString());
+            }
+        });
+    }
+
+    @Test
+    public void testUUID()
+    {
+        qt().forAll(TestUtils.bridges()).checkAssert(bridge -> {
+            assertTrue(bridge.uuid().deserialize(bridge.uuid().serialize(UUID.randomUUID())) instanceof UTF8String);
+            for (int test = 0; test < MAX_TESTS; test++)
+            {
+                UUID expected = UUID.randomUUID();
+                assertEquals(expected.toString(), bridge.uuid().deserialize(
+                        bridge.uuid().serialize(expected)).toString());
+            }
+        });
+    }
+
+    @Test
+    public void testLong()
+    {
+        qt().forAll(TestUtils.bridges()).checkAssert(bridge -> {
+            assertTrue(bridge.bigint().deserialize(bridge.bigint().serialize(Long.MAX_VALUE)) instanceof Long);
+            assertEquals(Long.MAX_VALUE, bridge.bigint().deserialize(
+                    ByteBuffer.allocate(8).putLong(0, Long.MAX_VALUE)));
+            qt().forAll(integers().all())
+                .checkAssert(integer -> assertEquals((long) integer, bridge.bigint().deserialize(
+                        bridge.bigint().serialize((long) integer))));
+            assertEquals(Long.MAX_VALUE, bridge.bigint().deserialize(bridge.bigint().serialize(Long.MAX_VALUE)));
+            assertEquals(Long.MIN_VALUE, bridge.bigint().deserialize(bridge.bigint().serialize(Long.MIN_VALUE)));
+            qt().withExamples(MAX_TESTS)
+                .forAll(longs().all())
+                .checkAssert(aLong -> assertEquals(aLong, bridge.bigint().deserialize(
+                        bridge.bigint().serialize(aLong))));
+        });
+    }
+
+    @Test
+    public void testDecimal()
+    {
+        qt().forAll(TestUtils.bridges()).checkAssert(bridge -> {
+            assertTrue(bridge.decimal().deserialize(
+                    bridge.decimal().serialize(BigDecimal.valueOf(500L))) instanceof Decimal);
+            assertEquals(Decimal.apply(500), bridge.decimal().deserialize(
+                    bridge.decimal().serialize(BigDecimal.valueOf(500L))));
+            assertNotSame(Decimal.apply(501), bridge.decimal().deserialize(
+                    bridge.decimal().serialize(BigDecimal.valueOf(500L))));
+            assertEquals(Decimal.apply(-1), bridge.decimal().deserialize(
+                    bridge.decimal().serialize(BigDecimal.valueOf(-1L))));
+            assertEquals(Decimal.apply(Long.MAX_VALUE), bridge.decimal().deserialize(
+                    bridge.decimal().serialize(BigDecimal.valueOf(Long.MAX_VALUE))));
+            assertEquals(Decimal.apply(Long.MIN_VALUE), bridge.decimal().deserialize(
+                    bridge.decimal().serialize(BigDecimal.valueOf(Long.MIN_VALUE))));
+            assertEquals(Decimal.apply(Integer.MAX_VALUE), bridge.decimal().deserialize(
+                    bridge.decimal().serialize(BigDecimal.valueOf(Integer.MAX_VALUE))));
+            assertEquals(Decimal.apply(Integer.MIN_VALUE), bridge.decimal().deserialize(
+                    bridge.decimal().serialize(BigDecimal.valueOf(Integer.MIN_VALUE))));
+            BigDecimal veryLargeValue = BigDecimal.valueOf(Integer.MAX_VALUE).multiply(BigDecimal.valueOf(5));
+            assertEquals(Decimal.apply(veryLargeValue), bridge.decimal().deserialize(
+                    bridge.decimal().serialize(veryLargeValue)));
+            qt().withExamples(MAX_TESTS)
+                .forAll(bigDecimals().ofBytes(128).withScale(10))
+                .checkAssert(decimal -> assertEquals(Decimal.apply(decimal), bridge.decimal().deserialize(
+                        bridge.decimal().serialize(decimal))));
+        });
+    }
+
+    @Test
+    public void testFloat()
+    {
+        qt().forAll(TestUtils.bridges()).checkAssert(bridge -> {
+            assertTrue(bridge.aFloat().deserialize(bridge.aFloat().serialize(Float.MAX_VALUE)) instanceof Float);
+            assertEquals(Float.MAX_VALUE, bridge.aFloat().deserialize(
+                    ByteBuffer.allocate(4).putFloat(0, Float.MAX_VALUE)));
+            qt().forAll(integers().all())
+                .checkAssert(integer -> assertEquals((float) integer, bridge.aFloat().deserialize(
+                        bridge.aFloat().serialize((float) integer))));
+            assertEquals(Float.MAX_VALUE, bridge.aFloat().deserialize(bridge.aFloat().serialize(Float.MAX_VALUE)));
+            assertEquals(Float.MIN_VALUE, bridge.aFloat().deserialize(bridge.aFloat().serialize(Float.MIN_VALUE)));
+            qt().withExamples(MAX_TESTS)
+                .forAll(floats().any())
+                .checkAssert(aFloat -> assertEquals(aFloat, bridge.aFloat().deserialize(
+                        bridge.aFloat().serialize(aFloat))));
+        });
+    }
+
+    @Test
+    public void testDouble()
+    {
+        qt().forAll(TestUtils.bridges()).checkAssert(bridge -> {
+            assertTrue(bridge.aDouble().deserialize(bridge.aDouble().serialize(Double.MAX_VALUE)) instanceof Double);
+            assertEquals(Double.MAX_VALUE, bridge.aDouble().deserialize(
+                    ByteBuffer.allocate(8).putDouble(0, Double.MAX_VALUE)));
+            qt().forAll(integers().all())
+                .checkAssert(integer -> assertEquals((double) integer, bridge.aDouble().deserialize(
+                        bridge.aDouble().serialize((double) integer))));
+            assertEquals(Double.MAX_VALUE, bridge.aDouble().deserialize(bridge.aDouble().serialize(Double.MAX_VALUE)));
+            assertEquals(Double.MIN_VALUE, bridge.aDouble().deserialize(bridge.aDouble().serialize(Double.MIN_VALUE)));
+            qt().withExamples(MAX_TESTS)
+                .forAll(doubles().any())
+                .checkAssert(aDouble -> assertEquals(aDouble, bridge.aDouble().deserialize(
+                        bridge.aDouble().serialize(aDouble))));
+        });
+    }
+
+    @Test
+    public void testAscii()
+    {
+        qt().forAll(TestUtils.bridges()).checkAssert(bridge -> {
+            assertTrue(bridge.ascii().deserialize(bridge.ascii().serialize("abc")) instanceof UTF8String);
+            qt().withExamples(MAX_TESTS)
+                .forAll(strings().ascii().ofLengthBetween(0, 100))
+                .checkAssert(string -> assertEquals(string, bridge.ascii().deserialize(
+                        bridge.ascii().serialize(string)).toString()));
+        });
+    }
+
+    @Test
+    public void testText()
+    {
+        qt().forAll(TestUtils.bridges()).checkAssert(bridge -> {
+            assertTrue(bridge.text().deserialize(bridge.text().serialize("abc")) instanceof UTF8String);
+            qt().withExamples(MAX_TESTS)
+                .forAll(strings().ascii().ofLengthBetween(0, 100))
+                .checkAssert(string -> assertEquals(string, bridge.text().deserialize(
+                        bridge.text().serialize(string)).toString()));
+            qt().withExamples(MAX_TESTS)
+                .forAll(strings().basicLatinAlphabet().ofLengthBetween(0, 100))
+                .checkAssert(string -> assertEquals(string, bridge.text().deserialize(
+                        bridge.text().serialize(string)).toString()));
+            qt().withExamples(MAX_TESTS)
+                .forAll(strings().numeric())
+                .checkAssert(string -> assertEquals(string, bridge.text().deserialize(
+                        bridge.text().serialize(string)).toString()));
+        });
+    }
+
+    @Test
+    public void testVarchar()
+    {
+        qt().forAll(TestUtils.bridges()).checkAssert(bridge -> {
+            assertTrue(bridge.varchar().deserialize(bridge.varchar().serialize("abc")) instanceof UTF8String);
+            qt().withExamples(MAX_TESTS)
+                .forAll(strings().ascii().ofLengthBetween(0, 100))
+                .checkAssert(string -> assertEquals(string, bridge.varchar().deserialize(
+                        bridge.varchar().serialize(string)).toString()));
+            qt().withExamples(MAX_TESTS)
+                .forAll(strings().basicLatinAlphabet().ofLengthBetween(0, 100))
+                .checkAssert(string -> assertEquals(string, bridge.varchar().deserialize(
+                        bridge.varchar().serialize(string)).toString()));
+            qt().withExamples(MAX_TESTS)
+                .forAll(strings().numeric())
+                .checkAssert(string -> assertEquals(string, bridge.varchar().deserialize(
+                        bridge.varchar().serialize(string)).toString()));
+        });
+    }
+
+    @Test
+    public void testInet()
+    {
+        qt().forAll(TestUtils.bridges()).checkAssert(bridge -> {
+            assertTrue(bridge.inet().deserialize(bridge.inet().serialize(RandomUtils.randomInet())) instanceof byte[]);
+            for (int test = 0; test < MAX_TESTS; test++)
+            {
+                InetAddress expected = RandomUtils.randomInet();
+                assertArrayEquals(expected.getAddress(), (byte[]) bridge.inet().deserialize(
+                        bridge.inet().serialize(expected)));
+            }
+        });
+    }
+
+    @Test
+    public void testDate()
+    {
+        qt().forAll(TestUtils.bridges()).checkAssert(bridge -> {
+            assertTrue(bridge.date().deserialize(bridge.date().serialize(5)) instanceof Integer);
+            qt().forAll(integers().all())
+                .checkAssert(integer -> assertEquals(integer - Integer.MIN_VALUE,
+                                                     bridge.date().deserialize(bridge.date().serialize(integer))));
+        });
+    }
+
+    @Test
+    public void testTime()
+    {
+        qt().forAll(TestUtils.bridges()).checkAssert(bridge -> {
+            assertTrue(bridge.time().deserialize(bridge.time().serialize(Long.MAX_VALUE)) instanceof Long);
+            qt().forAll(integers().all())
+                .checkAssert(integer -> assertEquals((long) integer, bridge.time().deserialize(
+                        bridge.time().serialize((long) integer))));
+            assertEquals(Long.MAX_VALUE, bridge.time().deserialize(bridge.time().serialize(Long.MAX_VALUE)));
+            assertEquals(Long.MIN_VALUE, bridge.time().deserialize(bridge.time().serialize(Long.MIN_VALUE)));
+            qt().withExamples(MAX_TESTS)
+                .forAll(longs().all())
+                .checkAssert(aLong -> assertEquals(aLong, bridge.time().deserialize(bridge.time().serialize(aLong))));
+        });
+    }
+
+    @Test
+    public void testTimestamp()
+    {
+        qt().forAll(TestUtils.bridges()).checkAssert(bridge -> {
+            Date now = new Date();
+            assertTrue(bridge.timestamp().deserialize(bridge.timestamp().serialize(now)) instanceof Long);
+            assertEquals(java.sql.Timestamp.from(now.toInstant()).getTime() * 1000L,
+                         bridge.timestamp().deserialize(bridge.timestamp().serialize(now)));
+            qt().withExamples(MAX_TESTS)
+                .forAll(dates().withMillisecondsBetween(0, Long.MAX_VALUE))
+                .checkAssert(date -> assertEquals(java.sql.Timestamp.from(date.toInstant()).getTime() * 1000L,
+                                                  bridge.timestamp().deserialize(bridge.timestamp().serialize(date))));
+        });
+    }
+
+    @Test
+    public void testBlob()
+    {
+        qt().forAll(TestUtils.bridges()).checkAssert(bridge -> {
+            assertTrue(bridge.blob().deserialize(
+                    bridge.blob().serialize(ByteBuffer.wrap(RandomUtils.randomBytes(5)))) instanceof byte[]);
+            for (int test = 0; test < MAX_TESTS; test++)
+            {
+                int size = RandomUtils.RANDOM.nextInt(1024);
+                byte[] expected = RandomUtils.randomBytes(size);
+                assertArrayEquals(expected, (byte[]) bridge.blob().deserialize(
+                        bridge.blob().serialize(ByteBuffer.wrap(expected))));
+            }
+        });
+    }
+
+    @Test
+    public void testEmpty()
+    {
+        qt().forAll(TestUtils.bridges()).checkAssert(bridge ->
+            assertNull(bridge.empty().deserialize(bridge.empty().serialize(null)))
+        );
+    }
+
+    @Test
+    public void testSmallInt()
+    {
+        qt().forAll(TestUtils.bridges()).checkAssert(bridge -> {
+            assertTrue(bridge.smallint().deserialize(bridge.smallint().serialize((short) 5)) instanceof Short);
+            qt().forAll(integers().between(Short.MIN_VALUE, Short.MAX_VALUE))
+                .checkAssert(integer -> {
+                    short expected = integer.shortValue();
+                    assertEquals(expected, bridge.smallint().deserialize(bridge.smallint().serialize(expected)));
+                });
+        });
+    }
+
+    @Test
+    public void testTinyInt()
+    {
+        qt().forAll(TestUtils.bridges()).checkAssert(bridge -> {
+            assertTrue(bridge.tinyint().deserialize(
+                    bridge.tinyint().serialize(RandomUtils.randomByte())) instanceof Byte);
+            for (int test = 0; test < MAX_TESTS; test++)
+            {
+                byte expected = RandomUtils.randomByte();
+                assertEquals(expected, bridge.tinyint().deserialize(bridge.tinyint().serialize(expected)));
+            }
+        });
+    }
+
+    @Test
+    public void testSerialization()
+    {
+        // CassandraBridge.serialize is mostly used for unit tests
+        qt().forAll(TestUtils.bridges()).checkAssert(bridge -> {
+            // BLOB,  VARINT
+            assertEquals("ABC", bridge.ascii().deserialize(bridge.ascii().serialize("ABC")).toString());
+            assertEquals(500L, bridge.bigint().deserialize(bridge.bigint().serialize(500L)));
+            assertEquals(true, bridge.bool().deserialize(bridge.bool().serialize(true)));
+            assertEquals(false, bridge.bool().deserialize(bridge.bool().serialize(false)));
+
+            byte[] bytes = new byte[]{'a', 'b', 'c', 'd'};
+            ByteBuffer buffer = bridge.blob().serialize(ByteBuffer.wrap(bytes));
+            byte[] result = new byte[4];
+            buffer.get(result);
+            assertArrayEquals(bytes, result);
+
+            assertEquals(500 + Integer.MIN_VALUE, bridge.date().deserialize(bridge.date().serialize(500)));
+            assertEquals(Decimal.apply(500000.2038484), bridge.decimal().deserialize(
+                    bridge.decimal().serialize(BigDecimal.valueOf(500000.2038484))));
+            assertEquals(123211.023874839, bridge.aDouble().deserialize(bridge.aDouble().serialize(123211.023874839)));
+            assertEquals(58383.23737832839f, bridge.aFloat().deserialize(
+                    bridge.aFloat().serialize(58383.23737832839f)));
+            try
+            {
+                assertEquals(InetAddress.getByName("www.apache.org"),
+                             InetAddress.getByAddress((byte[]) bridge.inet().deserialize(
+                                    bridge.inet().serialize(InetAddress.getByName("www.apache.org")))));
+            }
+            catch (UnknownHostException exception)
+            {
+                throw new RuntimeException(exception);
+            }
+            assertEquals(283848498, bridge.aInt().deserialize(bridge.aInt().serialize(283848498)));
+            assertEquals((short) 29, bridge.smallint().deserialize(bridge.smallint().serialize((short) 29)));
+            assertEquals("hello world", bridge.ascii().deserialize(bridge.text().serialize("hello world")).toString());
+            assertEquals(5002839L, bridge.time().deserialize(bridge.time().serialize(5002839L)));
+            Date now = new Date();
+            assertEquals(now.getTime() * 1000L, bridge.timestamp().deserialize(bridge.timestamp().serialize(now)));
+            UUID timeUuid = RandomUtils.getRandomTimeUUIDForTesting();
+            assertEquals(timeUuid, UUID.fromString(bridge.timeuuid().deserialize(
+                    bridge.timeuuid().serialize(timeUuid)).toString()));
+            assertEquals((byte) 100, bridge.tinyint().deserialize(bridge.tinyint().serialize((byte) 100)));
+            UUID uuid = UUID.randomUUID();
+            assertEquals(uuid, UUID.fromString(bridge.uuid().deserialize(bridge.uuid().serialize(uuid)).toString()));
+            assertEquals("ABCDEFG", bridge.varchar().deserialize(bridge.varchar().serialize("ABCDEFG")).toString());
+            assertEquals(Decimal.apply(12841924), bridge.varint().deserialize(
+                    bridge.varint().serialize(BigInteger.valueOf(12841924))));
+        });
+    }
+
+    @Test
+    public void testList()
+    {
+        runTest((partitioner, directory, bridge) ->
+            qt().forAll(TestUtils.cql3Type(bridge)).checkAssert(type -> {
+                CqlField.CqlList list = bridge.list(type);
+                List<Object> expected = IntStream.range(0, 128)
+                                                 .mapToObj(index -> RandomUtils.randomValue(type))
+                                                 .collect(Collectors.toList());
+                ByteBuffer buffer = list.serialize(expected);
+                List<Object> actual = Arrays.asList(((ArrayData) list.deserialize(buffer)).array());
+                assertEquals(expected.size(), actual.size());
+                for (int index = 0; index < expected.size(); index++)
+                {
+                    assertEquals(expected.get(index), type.toTestRowType(actual.get(index)));
+                }
+            }));
+    }
+
+    @Test
+    public void testSet()
+    {
+        runTest((partitioner, directory, bridge) ->
+            qt().forAll(TestUtils.cql3Type(bridge)).checkAssert(type -> {
+                CqlField.CqlSet set = bridge.set(type);
+                Set<Object> expected = IntStream.range(0, 128)
+                                                .mapToObj(integer -> RandomUtils.randomValue(type))
+                                                .collect(Collectors.toSet());
+                ByteBuffer buffer = set.serialize(expected);
+                Set<Object> actual = new HashSet<>(Arrays.asList(((ArrayData) set.deserialize(buffer)).array()));
+                assertEquals(expected.size(), actual.size());
+                for (Object value : actual)
+                {
+                    assertTrue(expected.contains(type.toTestRowType(value)));
+                }
+            }));
+    }
+
+    @Test
+    public void testMap()
+    {
+        runTest((partitioner, directory, bridge) ->
+            qt().forAll(TestUtils.cql3Type(bridge), TestUtils.cql3Type(bridge)).checkAssert((keyType, valueType) -> {
+                CqlField.CqlMap map = bridge.map(keyType, valueType);
+                int count = keyType.cardinality(128);
+                Map<Object, Object> expected = new HashMap<>(count);
+                for (int entry = 0; entry < count; entry++)
+                {
+                    Object key = null;
+                    while (key == null || expected.containsKey(key))
+                    {
+                        key = RandomUtils.randomValue(keyType);
+                    }
+                    expected.put(key, RandomUtils.randomValue(valueType));
+                }
+                ByteBuffer buffer = map.serialize(expected);
+                ArrayBasedMapData mapData = ((ArrayBasedMapData) map.deserialize(buffer));
+                ArrayData keys = mapData.keyArray();
+                ArrayData values = mapData.valueArray();
+                Map<Object, Object> actual = new HashMap<>(keys.numElements());
+                for (int index = 0; index < keys.numElements(); index++)
+                {
+                    Object key = keyType.toTestRowType(keys.get(index, keyType.sparkSqlType(BigNumberConfig.DEFAULT)));
+                    Object value = valueType.toTestRowType(values.get(index,
+                                                                      valueType.sparkSqlType(BigNumberConfig.DEFAULT)));
+                    actual.put(key, value);
+                }
+                assertEquals(expected.size(), actual.size());
+                for (Map.Entry<Object, Object> entry : expected.entrySet())
+                {
+                    assertEquals(entry.getValue(), actual.get(entry.getKey()));
+                }
+            }));
+    }
+
+    @Test
+    @SuppressWarnings("unchecked")
+    public void testUdts()
+    {
+        runTest((partitioner, directory, bridge) ->
+            qt().forAll(TestUtils.cql3Type(bridge), TestUtils.cql3Type(bridge)).checkAssert((firstType, secondType) -> {
+                CqlField.CqlUdt udt = bridge.udt("keyspace", "testudt")
+                                            .withField("a", firstType)
+                                            .withField("b", bridge.ascii())
+                                            .withField("c", secondType)
+                                            .build();
+                Map<String, Object> expected = (Map<String, Object>) RandomUtils.randomValue(udt);
+                assert expected != null;
+                ByteBuffer buffer = udt.serializeUdt(expected);
+                Map<String, Object> actual = udt.deserializeUdt(buffer, false);
+                assertEquals(expected.size(), actual.size());
+                for (Map.Entry<String, Object> entry : expected.entrySet())
+                {
+                    assertEquals(entry.getValue(),
+                                 udt.field(entry.getKey()).type().toTestRowType(actual.get(entry.getKey())));
+                }
+            }));
+    }
+
+    @Test
+    public void testTuples()
+    {
+        runTest((partitioner, directory, bridge) ->
+            qt().forAll(TestUtils.cql3Type(bridge), TestUtils.cql3Type(bridge)).checkAssert((firstType, secondType) -> {
+                CqlField.CqlTuple tuple = bridge.tuple(firstType,
+                                                       bridge.ascii(),
+                                                       secondType,
+                                                       bridge.timestamp(),
+                                                       bridge.uuid(),
+                                                       bridge.varchar());
+                Object[] expected = (Object[]) RandomUtils.randomValue(tuple);
+                assert expected != null;
+                ByteBuffer buffer = tuple.serializeTuple(expected);
+                Object[] actual = tuple.deserializeTuple(buffer, false);
+                assertEquals(expected.length, actual.length);
+                for (int index = 0; index < expected.length; index++)
+                {
+                    assertEquals(expected[index], tuple.type(index).toTestRowType(actual[index]));
+                }
+            }));
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/reader/OverlapTests.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/reader/OverlapTests.java
new file mode 100644
index 0000000..550bef0
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/reader/OverlapTests.java
@@ -0,0 +1,196 @@
+/*
+ * 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.cassandra.spark.reader;
+
+import java.math.BigInteger;
+
+import com.google.common.base.Preconditions;
+import org.junit.Test;
+
+import org.apache.cassandra.bridge.TokenRange;
+import org.apache.cassandra.spark.data.partitioner.Partitioner;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class OverlapTests
+{
+    @Test
+    public void testBasic()
+    {
+        SparkSSTableReader reader = reader(100L, 200L);
+        assertFalse(SparkSSTableReader.overlaps(reader, range(-10, -5)));
+        assertFalse(SparkSSTableReader.overlaps(reader, range(0, 0)));
+        assertFalse(SparkSSTableReader.overlaps(reader, range(-1, -1)));
+        assertFalse(SparkSSTableReader.overlaps(reader, range(50, 55)));
+        assertTrue(SparkSSTableReader.overlaps(reader, range(95, 100)));
+        assertTrue(SparkSSTableReader.overlaps(reader, range(100, 100)));
+        assertTrue(SparkSSTableReader.overlaps(reader, range(100, 105)));
+        assertTrue(SparkSSTableReader.overlaps(reader, range(100, 150)));
+        assertTrue(SparkSSTableReader.overlaps(reader, range(160, 200)));
+        assertTrue(SparkSSTableReader.overlaps(reader, range(200, 200)));
+        assertTrue(SparkSSTableReader.overlaps(reader, range(200, 205)));
+        assertFalse(SparkSSTableReader.overlaps(reader, range(201, 205)));
+        assertFalse(SparkSSTableReader.overlaps(reader, range(500, 550)));
+        assertTrue(SparkSSTableReader.overlaps(reader, range(50, 250)));
+        assertTrue(SparkSSTableReader.overlaps(reader, range(Partitioner.Murmur3Partitioner.minToken(),
+                                                             Partitioner.Murmur3Partitioner.maxToken())));
+    }
+
+    @Test
+    public void testMurmur3()
+    {
+        SparkSSTableReader murmur3Reader = reader(Partitioner.Murmur3Partitioner.minToken(),
+                                                  Partitioner.Murmur3Partitioner.maxToken());
+        assertTrue(SparkSSTableReader.overlaps(murmur3Reader, range(-10, -5)));
+        assertTrue(SparkSSTableReader.overlaps(murmur3Reader, range(0, 0)));
+        assertTrue(SparkSSTableReader.overlaps(murmur3Reader, range(300, 400)));
+        assertTrue(SparkSSTableReader.overlaps(murmur3Reader, range(Partitioner.Murmur3Partitioner.minToken(),
+                                                                    Partitioner.Murmur3Partitioner.minToken())));
+        assertTrue(SparkSSTableReader.overlaps(murmur3Reader, range(Partitioner.Murmur3Partitioner.minToken(),
+                                                                    Partitioner.Murmur3Partitioner.maxToken())));
+        assertTrue(SparkSSTableReader.overlaps(murmur3Reader, range(Partitioner.Murmur3Partitioner.maxToken(),
+                                                                    Partitioner.Murmur3Partitioner.maxToken())));
+        assertTrue(SparkSSTableReader.overlaps(murmur3Reader, range(Partitioner.Murmur3Partitioner.minToken().add(BigInteger.ONE),
+                                                                    Partitioner.Murmur3Partitioner.maxToken().subtract(BigInteger.ONE))));
+    }
+
+    @Test
+    public void testRandom()
+    {
+        SparkSSTableReader randomReader = reader(Partitioner.RandomPartitioner.minToken(),
+                                                 Partitioner.RandomPartitioner.maxToken());
+        assertFalse(SparkSSTableReader.overlaps(randomReader, range(-10, -5)));
+        assertTrue(SparkSSTableReader.overlaps(randomReader, range(0, 0)));
+        assertTrue(SparkSSTableReader.overlaps(randomReader, range(5000, 6000)));
+        assertTrue(SparkSSTableReader.overlaps(randomReader, range(Partitioner.RandomPartitioner.minToken(),
+                                                                   Partitioner.RandomPartitioner.minToken())));
+        assertTrue(SparkSSTableReader.overlaps(randomReader, range(Partitioner.RandomPartitioner.minToken(),
+                                                                   Partitioner.RandomPartitioner.maxToken())));
+        assertTrue(SparkSSTableReader.overlaps(randomReader, range(Partitioner.RandomPartitioner.maxToken(),
+                                                                   Partitioner.RandomPartitioner.maxToken())));
+        assertTrue(SparkSSTableReader.overlaps(randomReader, range(Partitioner.RandomPartitioner.minToken().add(BigInteger.ONE),
+                                                                   Partitioner.RandomPartitioner.maxToken().subtract(BigInteger.ONE))));
+    }
+
+    @Test
+    public void testMinEdge()
+    {
+        SparkSSTableReader minReader = reader(Partitioner.Murmur3Partitioner.minToken(),
+                                              BigInteger.valueOf(-6661324248839560306L));
+        assertTrue(SparkSSTableReader.overlaps(minReader, range(Partitioner.Murmur3Partitioner.minToken(),
+                                                                Partitioner.Murmur3Partitioner.minToken())));
+        assertTrue(SparkSSTableReader.overlaps(minReader, range(Partitioner.Murmur3Partitioner.minToken(),
+                                                                BigInteger.valueOf(-8198552921648689608L))));
+        assertTrue(SparkSSTableReader.overlaps(minReader, range(Partitioner.Murmur3Partitioner.minToken().subtract(BigInteger.ONE),
+                                                                BigInteger.valueOf(-8198552921648689608L))));
+        assertTrue(SparkSSTableReader.overlaps(minReader, range(Partitioner.Murmur3Partitioner.minToken().subtract(BigInteger.TEN),
+                                                                BigInteger.valueOf(-7173733806442603407L))));
+        assertTrue(SparkSSTableReader.overlaps(minReader, range(-7173733806442603407L, -6148914691236517207L)));
+        assertTrue(SparkSSTableReader.overlaps(minReader, range(-6661324248839560307L, -6661324248839560306L)));
+        assertTrue(SparkSSTableReader.overlaps(minReader, range(-6661324248839560306L, -6661324248839560306L)));
+        assertTrue(SparkSSTableReader.overlaps(minReader, range(-6661324248839560307L, -6148914691236517206L)));
+        assertFalse(SparkSSTableReader.overlaps(minReader, range(-6661324248839560305L, -6661324248839560305L)));
+        assertFalse(SparkSSTableReader.overlaps(minReader, range(-4611686018427387904L, -2562047788015215503L)));
+        assertFalse(SparkSSTableReader.overlaps(minReader, range(0L, 0L)));
+        assertFalse(SparkSSTableReader.overlaps(minReader, range(512409557603043100L, 8710962479251732707L)));
+        assertFalse(SparkSSTableReader.overlaps(minReader, range(Partitioner.Murmur3Partitioner.maxToken(),
+                                                                 Partitioner.Murmur3Partitioner.maxToken())));
+    }
+
+    @Test
+    public void testMaxEdge()
+    {
+        SparkSSTableReader maxReader = reader(BigInteger.valueOf(2049638230412172401L),
+                                              Partitioner.Murmur3Partitioner.maxToken());
+        assertFalse(SparkSSTableReader.overlaps(maxReader, range(Partitioner.Murmur3Partitioner.minToken(),
+                                                                 Partitioner.Murmur3Partitioner.minToken())));
+        assertFalse(SparkSSTableReader.overlaps(maxReader, range(Partitioner.Murmur3Partitioner.minToken(),
+                                                                 Partitioner.Murmur3Partitioner.minToken().add(BigInteger.TEN))));
+        assertFalse(SparkSSTableReader.overlaps(maxReader, range(-3074457345618258603L, -1537228672809129302L)));
+        assertFalse(SparkSSTableReader.overlaps(maxReader, range(-512409557603043101L, 0L)));
+        assertFalse(SparkSSTableReader.overlaps(maxReader, range(-512409557603043101L, 1024819115206086200L)));
+        assertFalse(SparkSSTableReader.overlaps(maxReader, range(512409557603043100L, 1537228672809129301L)));
+        assertTrue(SparkSSTableReader.overlaps(maxReader, range(2049638230412172400L, 2049638230412172401L)));
+        assertTrue(SparkSSTableReader.overlaps(maxReader, range(2049638230412172401L, 2049638230412172401L)));
+        assertTrue(SparkSSTableReader.overlaps(maxReader, range(2049638230412172402L, 2049638230412172402L)));
+        assertTrue(SparkSSTableReader.overlaps(maxReader, range(2049638230412172401L, 5636505133633474104L)));
+        assertTrue(SparkSSTableReader.overlaps(maxReader, range(BigInteger.valueOf(2049638230412172401L),
+                                                                Partitioner.Murmur3Partitioner.maxToken())));
+        assertTrue(SparkSSTableReader.overlaps(maxReader, range(BigInteger.valueOf(6661324248839560305L),
+                                                                Partitioner.Murmur3Partitioner.maxToken())));
+        assertTrue(SparkSSTableReader.overlaps(maxReader, range(Partitioner.Murmur3Partitioner.maxToken(),
+                                                                Partitioner.Murmur3Partitioner.maxToken())));
+    }
+
+    @Test
+    public void testZeroWrap()
+    {
+        SparkSSTableReader reader = reader(-1537228672809129302L, 1537228672809129301L);
+        assertFalse(SparkSSTableReader.overlaps(reader, range(Partitioner.Murmur3Partitioner.minToken(),
+                                                              Partitioner.Murmur3Partitioner.minToken())));
+        assertFalse(SparkSSTableReader.overlaps(reader, range(-5636505133633474105L, -2562047788015215503L)));
+        assertFalse(SparkSSTableReader.overlaps(reader, range(-1537228672809129303L, -1537228672809129303L)));
+        assertTrue(SparkSSTableReader.overlaps(reader, range(-1537228672809129302L, -1537228672809129302L)));
+        assertTrue(SparkSSTableReader.overlaps(reader, range(-1537228672809129301L, -1537228672809129301L)));
+        assertTrue(SparkSSTableReader.overlaps(reader, range(-1537228672809129302L, 0)));
+        assertTrue(SparkSSTableReader.overlaps(reader, range(0, 0)));
+        assertTrue(SparkSSTableReader.overlaps(reader, range(0, 1024819115206086200L)));
+        assertTrue(SparkSSTableReader.overlaps(reader, range(0, 1537228672809129301L)));
+        assertTrue(SparkSSTableReader.overlaps(reader, range(1537228672809129301L, 1537228672809129301L)));
+    }
+
+    private static TokenRange range(long start, long end)
+    {
+        return range(BigInteger.valueOf(start), BigInteger.valueOf(end));
+    }
+
+    private static TokenRange range(BigInteger start, BigInteger end)
+    {
+        return TokenRange.closed(start, end);
+    }
+
+    private static SparkSSTableReader reader(long start, long end)
+    {
+        return reader(BigInteger.valueOf(start), BigInteger.valueOf(end));
+    }
+
+    private static SparkSSTableReader reader(BigInteger start, BigInteger end)
+    {
+        Preconditions.checkArgument(start.compareTo(end) <= 0, "Start token must be less than end token");
+        return new SparkSSTableReader()
+        {
+            public BigInteger firstToken()
+            {
+                return start;
+            }
+
+            public BigInteger lastToken()
+            {
+                return end;
+            }
+
+            public boolean ignore()
+            {
+                return false;
+            }
+        };
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/reader/PartitionKeyTests.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/reader/PartitionKeyTests.java
new file mode 100644
index 0000000..79cba2a
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/reader/PartitionKeyTests.java
@@ -0,0 +1,77 @@
+/*
+ * 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.cassandra.spark.reader;
+
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.util.AbstractMap;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import com.google.common.collect.ImmutableList;
+import org.junit.Test;
+
+import org.apache.cassandra.spark.data.CqlField;
+import org.apache.cassandra.spark.data.CqlTable;
+
+import static org.apache.cassandra.spark.TestUtils.runTest;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class PartitionKeyTests
+{
+    @Test
+    public void testSinglePartitionKey()
+    {
+        runTest((partitioner, directory, bridge) -> {
+            List<CqlField> singlePartitionKey = Collections.singletonList(new CqlField(true, false, false, "a", bridge.aInt(), 0));
+            CqlTable table = mock(CqlTable.class);
+            when(table.partitionKeys()).thenReturn(singlePartitionKey);
+
+            ByteBuffer key = ByteBuffer.wrap(new byte[]{0, 0, 0, 1});
+            AbstractMap.SimpleEntry<ByteBuffer, BigInteger> actualKey = bridge.getPartitionKey(table, partitioner, ImmutableList.of("1"));
+
+            assertEquals(key, actualKey.getKey());
+            assertEquals(bridge.hash(partitioner, key), actualKey.getValue());
+            assertNotEquals(bridge.aInt().serialize(2), actualKey.getKey());
+        });
+    }
+
+    @Test
+    public void testMultiplePartitionKey()
+    {
+        runTest((partitioner, directory, bridge) -> {
+            List<CqlField> multiplePartitionKey = Arrays.asList(new CqlField(true, false, false, "a", bridge.aInt(), 0),
+                                                                new CqlField(true, false, false, "b", bridge.bigint(), 1),
+                                                                new CqlField(true, false, false, "c", bridge.text(), 2));
+            CqlTable table = mock(CqlTable.class);
+            when(table.partitionKeys()).thenReturn(multiplePartitionKey);
+
+            ByteBuffer key = ByteBuffer.wrap(new byte[]{0, 4, 0, 0, 0, 3, 0, 0, 8, 0, 0, 0, 0, 0, 0, 0, 1, 0, 0, 3, 120, 121, 122, 0});
+            AbstractMap.SimpleEntry<ByteBuffer, BigInteger> actualKey = bridge.getPartitionKey(table, partitioner, ImmutableList.of("3", "1", "xyz"));
+
+            assertEquals(key, actualKey.getKey());
+            assertEquals(bridge.hash(partitioner, key), actualKey.getValue());
+        });
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/reader/RidTests.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/reader/RidTests.java
new file mode 100644
index 0000000..6378a27
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/reader/RidTests.java
@@ -0,0 +1,136 @@
+/*
+ * 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.cassandra.spark.reader;
+
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+public class RidTests
+{
+    @Test
+    public void testInit()
+    {
+        Rid rid = new Rid();
+        assertNull(rid.getPartitionKey());
+        assertNull(rid.getColumnName());
+        assertNull(rid.getValue());
+        assertFalse(rid.isNewPartition);
+    }
+
+    @Test
+    public void testSetPartitionKey()
+    {
+        Rid rid = new Rid();
+        rid.setPartitionKeyCopy(ByteBuffer.wrap("101".getBytes()), BigInteger.ZERO);
+        assertNotNull(rid.getPartitionKey());
+        assertNull(rid.getColumnName());
+        assertNull(rid.getValue());
+
+        assertTrue(rid.isNewPartition);
+        assertTrue(rid.isNewPartition());
+        assertFalse(rid.isNewPartition);
+        assertEquals("101", toString(rid.getPartitionKey()));
+
+        rid.setPartitionKeyCopy(ByteBuffer.wrap("102".getBytes()), BigInteger.ZERO);
+        assertTrue(rid.isNewPartition);
+        assertTrue(rid.isNewPartition());
+        assertFalse(rid.isNewPartition);
+        assertEquals("102", toString(rid.getPartitionKey()));
+    }
+
+    @Test
+    public void testSetColumnKey()
+    {
+        Rid rid = new Rid();
+        assertNull(rid.getPartitionKey());
+        assertNull(rid.getColumnName());
+        assertNull(rid.getValue());
+        rid.setColumnNameCopy(ByteBuffer.wrap("101".getBytes()));
+
+        assertNull(rid.getPartitionKey());
+        assertNotNull(rid.getColumnName());
+        assertNull(rid.getValue());
+        assertEquals("101", toString(rid.getColumnName()));
+    }
+
+    @Test
+    public void testSetValue()
+    {
+        Rid rid = new Rid();
+        assertNull(rid.getPartitionKey());
+        assertNull(rid.getColumnName());
+        assertNull(rid.getValue());
+        rid.setValueCopy(ByteBuffer.wrap("101".getBytes()));
+
+        assertNull(rid.getPartitionKey());
+        assertNull(rid.getColumnName());
+        assertNotNull(rid.getValue());
+        assertEquals("101", toString(rid.getValue()));
+    }
+
+    @Test
+    public void testSetAll()
+    {
+        Rid rid = new Rid();
+        assertNull(rid.getPartitionKey());
+        assertNull(rid.getColumnName());
+        assertNull(rid.getValue());
+        rid.setPartitionKeyCopy(ByteBuffer.wrap("101".getBytes()), BigInteger.ZERO);
+        rid.setColumnNameCopy(ByteBuffer.wrap("102".getBytes()));
+        rid.setValueCopy(ByteBuffer.wrap("103".getBytes()));
+
+        assertTrue(rid.isNewPartition);
+        assertTrue(rid.isNewPartition());
+        assertFalse(rid.isNewPartition);
+
+        assertNotNull(rid.getPartitionKey());
+        assertNotNull(rid.getColumnName());
+        assertNotNull(rid.getValue());
+        assertEquals("101", toString(rid.getPartitionKey()));
+        assertEquals("102", toString(rid.getColumnName()));
+        assertEquals("103", toString(rid.getValue()));
+
+        rid.setPartitionKeyCopy(ByteBuffer.wrap("104".getBytes()), BigInteger.ZERO);
+        assertTrue(rid.isNewPartition);
+        assertTrue(rid.isNewPartition());
+        assertFalse(rid.isNewPartition);
+        rid.setColumnNameCopy(ByteBuffer.wrap("105".getBytes()));
+        rid.setValueCopy(ByteBuffer.wrap("106".getBytes()));
+        assertEquals("104", toString(rid.getPartitionKey()));
+        assertEquals("105", toString(rid.getColumnName()));
+        assertEquals("106", toString(rid.getValue()));
+    }
+
+    private static String toString(ByteBuffer buffer)
+    {
+        byte[] bytes = new byte[buffer.remaining()];
+        buffer.get(bytes);
+        return new String(bytes, StandardCharsets.UTF_8);
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/reader/SchemaTests.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/reader/SchemaTests.java
new file mode 100644
index 0000000..955fca7
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/reader/SchemaTests.java
@@ -0,0 +1,584 @@
+/*
+ * 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.cassandra.spark.reader;
+
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.regex.Matcher;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import org.junit.Test;
+
+import org.apache.cassandra.bridge.CassandraBridge;
+import org.apache.cassandra.bridge.CassandraVersion;
+import org.apache.cassandra.spark.TestUtils;
+import org.apache.cassandra.spark.data.CqlField;
+import org.apache.cassandra.spark.data.CqlTable;
+import org.apache.cassandra.spark.data.ReplicationFactor;
+import org.apache.cassandra.spark.data.VersionRunner;
+import org.apache.cassandra.spark.data.partitioner.Partitioner;
+import org.jetbrains.annotations.Nullable;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNotSame;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertTrue;
+import static org.quicktheories.QuickTheory.qt;
+
+public class SchemaTests extends VersionRunner
+{
+    public static final String SCHEMA = "CREATE TABLE backup_test.sbr_test (\n"
+                                      + "    account_id uuid,\n"
+                                      + "    balance bigint,\n"
+                                      + "    name text,\n"
+                                      + "    PRIMARY KEY(account_id)\n"
+                                      + ") WITH bloom_filter_fp_chance = 0.1\n"
+                                      + "    AND caching = {'keys': 'ALL', 'rows_per_partition': 'NONE'}\n"
+                                      + "    AND comment = 'Created by: jberragan'\n"
+                                      + "    AND compaction = {'class': 'org.apache.cassandra.db.compaction.LeveledCompactionStrategy'}\n"
+                                      + "    AND compression = {'chunk_length_in_kb': '64', "
+                                      +                        "'class': 'org.apache.cassandra.io.compress.LZ4Compressor'}\n"
+                                      + "    AND crc_check_chance = 1.0\n"
+                                      + "    AND default_time_to_live = 0\n"
+                                      + "    AND gc_grace_seconds = 864000\n"
+                                      + "    AND max_index_interval = 2048\n"
+                                      + "    AND memtable_flush_period_in_ms = 0\n"
+                                      + "    AND min_index_interval = 128\n;";
+
+    public SchemaTests(CassandraVersion version)
+    {
+        super(version);
+    }
+
+    @Test
+    public void testBuild()
+    {
+        ReplicationFactor replicationFactor = new ReplicationFactor(
+                ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy, ImmutableMap.of("DC1", 3, "DC2", 3));
+        CqlTable table = bridge.buildSchema(SCHEMA, "backup_test", replicationFactor);
+        List<CqlField> fields = table.fields();
+        assertNotNull(fields);
+        assertEquals(3, fields.size());
+        assertEquals("account_id", fields.get(0).name());
+        assertEquals("balance", fields.get(1).name());
+        assertEquals("name", fields.get(2).name());
+        assertEquals(SCHEMA, table.createStatement());
+        assertEquals(3, table.replicationFactor().getOptions().get("DC1").intValue());
+        assertEquals(3, table.replicationFactor().getOptions().get("DC2").intValue());
+        assertNull(table.replicationFactor().getOptions().get("DC3"));
+        assertEquals(1, table.numPartitionKeys());
+        assertEquals(0, table.numClusteringKeys());
+        assertEquals(0, table.numStaticColumns());
+        assertEquals(2, table.numValueColumns());
+    }
+
+    @Test
+    public void testEquality()
+    {
+        ReplicationFactor replicationFactor = new ReplicationFactor(
+                ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy, ImmutableMap.of("DC1", 3, "DC2", 3));
+        CqlTable table1 = bridge.buildSchema(SCHEMA, "backup_test", replicationFactor);
+        CqlTable table2 = bridge.buildSchema(SCHEMA, "backup_test", replicationFactor);
+        assertNotSame(table1, table2);
+        assertNotEquals(null, table2);
+        assertNotEquals(null, table1);
+        assertNotEquals(new ArrayList<>(), table1);
+        assertEquals(table1, table1);
+        assertEquals(table1, table2);
+        assertEquals(table1.hashCode(), table2.hashCode());
+    }
+
+    @Test
+    public void testSameKeyspace()
+    {
+        ReplicationFactor replicationFactor = new ReplicationFactor(
+                ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy, ImmutableMap.of("DC1", 3, "DC2", 3));
+        CqlTable table1 = bridge.buildSchema(SCHEMA, "backup_test", replicationFactor);
+        CqlTable table2 = bridge.buildSchema(SCHEMA.replace("sbr_test", "sbr_test2"), "backup_test", replicationFactor);
+        assertNotSame(table1, table2);
+        assertEquals("sbr_test2", table2.table());
+        assertEquals("sbr_test", table1.table());
+    }
+
+    @Test
+    public void testHasher()
+    {
+        // Casts to (ByteBuffer) required when compiling with Java 8
+        assertEquals(BigInteger.valueOf(6747049197585865300L),
+                     bridge.hash(Partitioner.Murmur3Partitioner, (ByteBuffer) ByteBuffer.allocate(8).putLong(992393994949L).flip()));
+        assertEquals(BigInteger.valueOf(7071430368280192841L),
+                     bridge.hash(Partitioner.Murmur3Partitioner, (ByteBuffer) ByteBuffer.allocate(4).putInt(999).flip()));
+        assertEquals(new BigInteger("28812675363873787366858706534556752548"),
+                     bridge.hash(Partitioner.RandomPartitioner, (ByteBuffer) ByteBuffer.allocate(8).putLong(34828288292L).flip()));
+        assertEquals(new BigInteger("154860613751552680515987154638148676974"),
+                     bridge.hash(Partitioner.RandomPartitioner, (ByteBuffer) ByteBuffer.allocate(4).putInt(1929239).flip()));
+    }
+
+    @Test
+    public void testUUID()
+    {
+        assertEquals(1, bridge.getTimeUUID().version());
+    }
+
+    @Test
+    public void testCollections()
+    {
+        String createStatement = "CREATE TABLE backup_test.collection_test (account_id uuid PRIMARY KEY, balance bigint, names set<text>);";
+        ReplicationFactor replicationFactor = new ReplicationFactor(
+                ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy, ImmutableMap.of("DC1", 3, "DC2", 3));
+        CqlTable table = bridge.buildSchema(createStatement, "backup_test", replicationFactor);
+        assertEquals(CqlField.CqlType.InternalType.Set, table.getField("names").type().internalType());
+    }
+
+    @Test
+    public void testSetClusteringKey()
+    {
+        String createStatement = "CREATE TABLE backup_test.sbr_test_set_ck (pk uuid, ck frozen<set<text>>, PRIMARY KEY (pk, ck));";
+        ReplicationFactor replicationFactor = new ReplicationFactor(
+                ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy, ImmutableMap.of("DC1", 3, "DC2", 3));
+        bridge.buildSchema(createStatement, "backup_test", replicationFactor);
+    }
+
+    @Test
+    public void testListClusteringKey()
+    {
+        String createStatement = "CREATE TABLE backup_test.sbr_test_list_ck (pk uuid, ck frozen<list<bigint>>, PRIMARY KEY (pk, ck));";
+        ReplicationFactor replicationFactor = new ReplicationFactor(
+                ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy, ImmutableMap.of("DC1", 3, "DC2", 3));
+        bridge.buildSchema(createStatement, "backup_test", replicationFactor);
+    }
+
+    @Test
+    public void testMapClusteringKey()
+    {
+        String createStatement = "CREATE TABLE backup_test.sbr_test_map_ck (pk uuid, ck frozen<map<uuid, timestamp>>, PRIMARY KEY (pk, ck));";
+        ReplicationFactor replicationFactor = new ReplicationFactor(
+                ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy, ImmutableMap.of("DC1", 3, "DC2", 3));
+        bridge.buildSchema(createStatement, "backup_test", replicationFactor);
+    }
+
+    @Test(expected = UnsupportedOperationException.class)
+    public void testNativeUnsupportedColumnMetaData()
+    {
+        String createStatement = "CREATE TABLE backup_test.sbr_test (account_id uuid, transactions counter, PRIMARY KEY(account_id));";
+        ReplicationFactor replicationFactor = new ReplicationFactor(
+                ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy, ImmutableMap.of("DC1", 3, "DC2", 3));
+        bridge.buildSchema(createStatement, "backup_test", replicationFactor);
+    }
+
+    @Test(expected = UnsupportedOperationException.class)
+    public void testUnsupportedInnerType()
+    {
+        String createStatement = "CREATE TABLE backup_test.sbr_test (account_id uuid, transactions counter, PRIMARY KEY(account_id));";
+        ReplicationFactor replicationFactor = new ReplicationFactor(
+                ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy, ImmutableMap.of("DC1", 3, "DC2", 3));
+        bridge.buildSchema(createStatement, "backup_test", replicationFactor);
+    }
+
+    @Test(expected = UnsupportedOperationException.class)
+    public void testUnsupportedUdt()
+    {
+        String createStatement = "CREATE TABLE backup_test.sbr_test (account_id uuid, transactions frozen<testudt>, PRIMARY KEY (account_id));";
+        ReplicationFactor replicationFactor = new ReplicationFactor(
+                ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy, ImmutableMap.of("DC1", 3, "DC2", 3));
+        bridge.buildSchema(createStatement, "backup_test", replicationFactor, Partitioner.Murmur3Partitioner,
+                ImmutableSet.of("CREATE TYPE backup_test.testudt(birthday timestamp, count bigint, length counter);"));
+    }
+
+    @Test
+    public void testCollectionMatcher()
+    {
+        qt().forAll(TestUtils.cql3Type(bridge)).checkAssert(type -> testMatcher("set<%s>", "set", type));
+        qt().forAll(TestUtils.cql3Type(bridge)).checkAssert(type -> testMatcher("list<%s>", "list", type));
+        qt().forAll(TestUtils.cql3Type(bridge), TestUtils.cql3Type(bridge)).checkAssert((first, second) -> {
+            testMatcher("map<%s,%s>", "map", first, second);
+            testMatcher("map<%s , %s>", "map", first, second);
+        });
+        qt().forAll(TestUtils.cql3Type(bridge)).checkAssert(type -> testMatcher(type.cqlName(), null, null));
+        qt().forAll(TestUtils.cql3Type(bridge), TestUtils.cql3Type(bridge)).checkAssert((first, second) -> {
+            testMatcher("tuple<%s,%s>", "tuple", first, second);
+            testMatcher("tuple<%s , %s>", "tuple", first, second);
+        });
+    }
+
+    private void testMatcher(String pattern, String collection, CqlField.NativeType type)
+    {
+        testMatcher(pattern, collection, type, null);
+    }
+
+    private void testMatcher(String pattern, String collection, CqlField.NativeType first, CqlField.NativeType second)
+    {
+        boolean isMap = second != null;
+        String string;
+        if (first == null && second == null)
+        {
+            string = pattern;
+        }
+        else if (second == null)
+        {
+            string = String.format(pattern, first);
+        }
+        else
+        {
+            string = String.format(pattern, first, second);
+        }
+
+        Matcher matcher = CassandraBridge.COLLECTION_PATTERN.matcher(string);
+        assertEquals(collection != null && first != null, matcher.matches());
+        if (matcher.matches())
+        {
+            assertNotNull(collection);
+            assertNotNull(first);
+            assertEquals(collection, matcher.group(1));
+            String[] types = CassandraBridge.splitInnerTypes(matcher.group(2));
+            assertEquals(first, bridge.nativeType(types[0].toUpperCase()));
+            if (isMap)
+            {
+                assertEquals(second, bridge.nativeType(types[1].toUpperCase()));
+            }
+        }
+        else
+        {
+            // Raw CQL3 data type
+            bridge.nativeType(pattern.toUpperCase());
+        }
+    }
+
+    @Test
+    public void testFrozenMatcher()
+    {
+        qt().forAll(TestUtils.cql3Type(bridge))
+            .checkAssert(type -> testFrozen("frozen<set<%s>>", CqlField.CqlSet.class, type));
+        qt().forAll(TestUtils.cql3Type(bridge))
+            .checkAssert(type -> testFrozen("frozen<list<%s>>", CqlField.CqlList.class, type));
+        qt().forAll(TestUtils.cql3Type(bridge), TestUtils.cql3Type(bridge)).checkAssert((first, second) -> {
+            testFrozen("frozen<map<%s,%s>>", CqlField.CqlMap.class, first, second);
+            testFrozen("frozen<map<%s , %s>>", CqlField.CqlMap.class, first, second);
+        });
+    }
+
+    @Test
+    public void testNestedFrozenSet()
+    {
+        String pattern = "map<text, frozen<set<bigint>>>";
+        CqlField.CqlType type = bridge.parseType(pattern);
+        assertNotNull(type);
+        assertTrue(type instanceof CqlField.CqlMap);
+        CqlField.CqlMap map = (CqlField.CqlMap) type;
+        assertTrue(map.keyType() instanceof CqlField.NativeType);
+        assertTrue(map.valueType() instanceof CqlField.CqlFrozen);
+        CqlField.NativeType key = (CqlField.NativeType) map.keyType();
+        assertSame(key, bridge.text());
+        CqlField.CqlFrozen value = (CqlField.CqlFrozen) map.valueType();
+        CqlField.CqlSet inner = (CqlField.CqlSet) value.inner();
+        assertSame(inner.type(), bridge.bigint());
+    }
+
+    @Test
+    public void testNestedFrozenMap()
+    {
+        String pattern = "map<text, frozen<map<bigint, text>>>";
+        CqlField.CqlType type = bridge.parseType(pattern);
+        assertNotNull(type);
+        assertTrue(type instanceof CqlField.CqlMap);
+        CqlField.CqlMap map = (CqlField.CqlMap) type;
+        assertTrue(map.keyType() instanceof CqlField.NativeType);
+        assertTrue(map.valueType() instanceof CqlField.CqlFrozen);
+        CqlField.NativeType key = (CqlField.NativeType) map.keyType();
+        assertSame(key, bridge.text());
+        CqlField.CqlFrozen value = (CqlField.CqlFrozen) map.valueType();
+        CqlField.CqlMap inner = (CqlField.CqlMap) value.inner();
+        assertSame(inner.keyType(), bridge.bigint());
+        assertSame(inner.valueType(), bridge.text());
+    }
+
+    private void testFrozen(String pattern,
+                            Class<? extends CqlField.CqlCollection> collectionType,
+                            CqlField.CqlType innerType)
+    {
+        testFrozen(pattern, collectionType, innerType, null);
+    }
+
+    private void testFrozen(String pattern,
+                            Class<? extends CqlField.CqlCollection> collectionType,
+                            CqlField.CqlType first,
+                            @Nullable CqlField.CqlType second)
+    {
+        pattern = second != null ? String.format(pattern, first, second) : String.format(pattern, first);
+        CqlField.CqlType type = bridge.parseType(pattern);
+        assertNotNull(type);
+        assertTrue(type instanceof CqlField.CqlFrozen);
+        CqlField.CqlFrozen frozen = (CqlField.CqlFrozen) type;
+        CqlField.CqlCollection inner = (CqlField.CqlCollection) frozen.inner();
+        assertNotNull(inner);
+        assertTrue(collectionType.isInstance(inner));
+        assertEquals(first, inner.type());
+        if (second != null)
+        {
+            CqlField.CqlMap map = (CqlField.CqlMap) inner;
+            assertEquals(second, map.valueType());
+        }
+    }
+
+    /* User-Defined Types */
+
+    @Test
+    public void testUdts()
+    {
+        ReplicationFactor replicationFactor = new ReplicationFactor(
+                ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy, ImmutableMap.of("DC1", 3, "DC2", 3));
+        String keyspace = "udt_keyspace";
+        String udtName = "udt_name";
+        CqlTable table = bridge.buildSchema("CREATE TABLE " + keyspace + ".udt_test (\n"
+                                          + "    account_id uuid PRIMARY KEY,\n"
+                                          + "    balance bigint,\n"
+                                          + "    info frozen<" + udtName + ">,\n"
+                                          + "    name text\n"
+                                          + ");", keyspace, replicationFactor, Partitioner.Murmur3Partitioner,
+                                            ImmutableSet.of("CREATE TYPE " + keyspace + "." + udtName + " (\n"
+                                                          + "  birthday timestamp,\n"
+                                                          + "  nationality text,\n"
+                                                          + "  weight float,\n"
+                                                          + "  height int\n"
+                                                          + ");"));
+        assertEquals(1, table.udts().size());
+        CqlField.CqlUdt udt = table.udts().stream().findFirst().get();
+        assertEquals(udtName, udt.name());
+        List<CqlField> udtFields = udt.fields();
+        assertEquals(4, udtFields.size());
+        assertEquals(bridge.timestamp(), udtFields.get(0).type());
+        assertEquals(bridge.text(), udtFields.get(1).type());
+        assertEquals(bridge.aFloat(), udtFields.get(2).type());
+        assertEquals(bridge.aInt(), udtFields.get(3).type());
+
+        List<CqlField> fields = table.fields();
+        assertEquals(bridge.uuid(), fields.get(0).type());
+        assertEquals(bridge.bigint(), fields.get(1).type());
+        assertEquals(CqlField.CqlType.InternalType.Frozen, fields.get(2).type().internalType());
+        assertEquals(bridge.text(), fields.get(3).type());
+
+        CqlField.CqlFrozen frozenField = (CqlField.CqlFrozen) fields.get(2).type();
+        assertEquals(CqlField.CqlType.InternalType.Udt, frozenField.inner().internalType());
+
+        CqlField.CqlUdt udtField = (CqlField.CqlUdt) frozenField.inner();
+        assertEquals(bridge.timestamp(), udtField.field(0).type());
+        assertEquals(bridge.text(), udtField.field(1).type());
+        assertEquals(bridge.aFloat(), udtField.field(2).type());
+        assertEquals(bridge.aInt(), udtField.field(3).type());
+    }
+
+    @Test
+    public void testCollectionUdts()
+    {
+        ReplicationFactor replicationFactor = new ReplicationFactor(
+                ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy, ImmutableMap.of("DC1", 3, "DC2", 3));
+        String keyspace = "collection_keyspace";
+        String udtName = "basic_info";
+        CqlTable table = bridge.buildSchema("CREATE TABLE " + keyspace + "." + udtName + " (\n"
+                                          + "    account_id uuid PRIMARY KEY,\n"
+                                          + "    balance bigint,\n"
+                                          + "    info frozen<map<text, " + udtName + ">>,\n"
+                                          + "    name text\n"
+                                          + ");", "collection_keyspace", replicationFactor, Partitioner.Murmur3Partitioner,
+                                            ImmutableSet.of("CREATE TYPE " + keyspace + "." + udtName + " (\n"
+                                                         + "  birthday timestamp,\n"
+                                                         + "  nationality text,\n"
+                                                         + "  weight float,\n"
+                                                         + "  height int\n"
+                                                         + ");"));
+        List<CqlField> fields = table.fields();
+        assertEquals(bridge.uuid(), fields.get(0).type());
+        assertEquals(bridge.bigint(), fields.get(1).type());
+        assertEquals(CqlField.CqlType.InternalType.Frozen, fields.get(2).type().internalType());
+        assertEquals(bridge.text(), fields.get(3).type());
+
+        CqlField.CqlMap mapField = (CqlField.CqlMap) ((CqlField.CqlFrozen) fields.get(2).type()).inner();
+        assertEquals(bridge.text(), mapField.keyType());
+        CqlField.CqlFrozen valueType = (CqlField.CqlFrozen) mapField.valueType();
+        CqlField.CqlUdt udtField = (CqlField.CqlUdt) valueType.inner();
+        assertEquals(bridge.timestamp(), udtField.field(0).type());
+        assertEquals(bridge.text(), udtField.field(1).type());
+        assertEquals(bridge.aFloat(), udtField.field(2).type());
+        assertEquals(bridge.aInt(), udtField.field(3).type());
+    }
+
+    @Test
+    public void testParseUdt()
+    {
+        ReplicationFactor replicationFactor = new ReplicationFactor(
+                ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy, ImmutableMap.of("DC1", 3, "DC2", 3));
+        CqlTable table = bridge.buildSchema(SCHEMA, "backup_test", replicationFactor, Partitioner.Murmur3Partitioner,
+                ImmutableSet.of("CREATE TYPE backup_test.tuple_test (a int, b bigint, c blob, d text)"));
+        assertEquals(1, table.udts().size());
+        CqlField.CqlUdt udt = table.udts().stream().findFirst().get();
+        assertEquals("tuple_test", udt.name());
+        List<CqlField> fields = udt.fields();
+        assertEquals(4, fields.size());
+        assertEquals(bridge.aInt(), fields.get(0).type());
+        assertEquals(bridge.bigint(), fields.get(1).type());
+        assertEquals(bridge.blob(), fields.get(2).type());
+        assertEquals(bridge.text(), fields.get(3).type());
+    }
+
+    @Test
+    public void testParseTuple()
+    {
+        ReplicationFactor replicationFactor = new ReplicationFactor(
+                ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy, ImmutableMap.of("DC1", 3, "DC2", 3));
+        CqlTable table = bridge.buildSchema("CREATE TABLE tuple_keyspace.tuple_test (\n"
+                                          + "    account_id uuid PRIMARY KEY,\n"
+                                          + "    balance bigint,\n"
+                                          + "    info tuple<bigint, text, float, boolean>,"
+                                          + "    name text\n"
+                                          + ")", "tuple_keyspace", replicationFactor, Partitioner.Murmur3Partitioner);
+        List<CqlField> fields = table.fields();
+        assertEquals(4, fields.size());
+        assertEquals(bridge.uuid(), fields.get(0).type());
+        assertEquals(bridge.bigint(), fields.get(1).type());
+        assertEquals(bridge.text(), fields.get(3).type());
+
+        assertEquals(CqlField.CqlType.InternalType.Frozen, fields.get(2).type().internalType());
+        CqlField.CqlTuple tuple = (CqlField.CqlTuple) ((CqlField.CqlFrozen) fields.get(2).type()).inner();
+        assertEquals(bridge.bigint(), tuple.type(0));
+        assertEquals(bridge.text(), tuple.type(1));
+        assertEquals(bridge.aFloat(), tuple.type(2));
+        assertEquals(bridge.bool(), tuple.type(3));
+    }
+
+    @Test
+    public void testComplexSchema()
+    {
+        String keyspace = "complex_schema1";
+        String type1 = "CREATE TYPE " + keyspace + ".field_with_timestamp (\n"
+                     + "    field text,\n"
+                     + "    \"timeWithZone\" frozen<" + keyspace + ".analytics_time_with_zone>\n"
+                     + ");";
+        String type2 = "CREATE TYPE " + keyspace + ".first_last_seen_fields_v1 (\n"
+                     + "    \"firstSeen\" frozen<" + keyspace + ".field_with_timestamp>,\n"
+                     + "    \"lastSeen\" frozen<" + keyspace + ".field_with_timestamp>,\n"
+                     + "    \"firstTransaction\" frozen<" + keyspace + ".field_with_timestamp>,\n"
+                     + "    \"lastTransaction\" frozen<" + keyspace + ".field_with_timestamp>,\n"
+                     + "    \"firstListening\" frozen<" + keyspace + ".field_with_timestamp>,\n"
+                     + "    \"lastListening\" frozen<" + keyspace + ".field_with_timestamp>,\n"
+                     + "    \"firstReading\" frozen<" + keyspace + ".field_with_timestamp>,\n"
+                     + "    \"lastReading\" frozen<" + keyspace + ".field_with_timestamp>,\n"
+                     + "    \"outputEvent\" text,\n"
+                     + "    \"eventHistory\" frozen<map<bigint, frozen<map<text, boolean>>>>\n"
+                     + ");";
+        String type3 = "CREATE TYPE " + keyspace + ".analytics_time_with_zone (\n"
+                     + "    time bigint,\n"
+                     + "    \"timezoneOffsetMinutes\" int\n"
+                     + ");";
+        String type4 = "CREATE TYPE " + keyspace + ".first_last_seen_dimensions_v1 (\n"
+                     + "    \"osMajorVersion\" text,\n"
+                     + "    \"storeFrontId\" text,\n"
+                     + "    platform text,\n"
+                     + "    time_range text\n"
+                     + ");";
+        String tableStr = "CREATE TABLE " + keyspace + ".books_ltd_v3 (\n"
+                        + "    \"consumerId\" text,\n"
+                        + "    dimensions frozen<" + keyspace + ".first_last_seen_dimensions_v1>,\n"
+                        + "    fields frozen<" + keyspace + ".first_last_seen_fields_v1>,\n"
+                        + "    first_transition_time frozen<" + keyspace + ".analytics_time_with_zone>,\n"
+                        + "    last_transition_time frozen<" + keyspace + ".analytics_time_with_zone>,\n"
+                        + "    prev_state_id text,\n"
+                        + "    state_id text,\n"
+                        + "    PRIMARY KEY (\"consumerId\", dimensions)\n"
+                        + ") WITH CLUSTERING ORDER BY (dimensions ASC);";
+        ReplicationFactor replicationFactor = new ReplicationFactor(
+                ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy, ImmutableMap.of("DC1", 3, "DC2", 3));
+        CqlTable table = bridge.buildSchema(tableStr, keyspace, replicationFactor, Partitioner.Murmur3Partitioner,
+                ImmutableSet.of(type1, type2, type3, type4));
+        assertEquals("books_ltd_v3", table.table());
+        assertEquals(keyspace, table.keyspace());
+        assertEquals(7, table.fields().size());
+        assertEquals(1, table.partitionKeys().size());
+        assertEquals(1, table.clusteringKeys().size());
+
+        List<CqlField> fields = table.fields();
+        assertEquals(7, fields.size());
+        assertEquals("consumerId", fields.get(0).name());
+        assertEquals(bridge.text(), fields.get(0).type());
+        CqlField clusteringKey = fields.get(1);
+        assertEquals("dimensions", clusteringKey.name());
+        assertEquals(CqlField.CqlType.InternalType.Frozen, clusteringKey.type().internalType());
+
+        CqlField.CqlUdt clusteringUDT = (CqlField.CqlUdt) ((CqlField.CqlFrozen) clusteringKey.type()).inner();
+        assertEquals("first_last_seen_dimensions_v1", clusteringUDT.name());
+        assertEquals(keyspace, clusteringUDT.keyspace());
+        assertEquals("osMajorVersion", clusteringUDT.field(0).name());
+        assertEquals(bridge.text(), clusteringUDT.field(0).type());
+        assertEquals("storeFrontId", clusteringUDT.field(1).name());
+        assertEquals(bridge.text(), clusteringUDT.field(1).type());
+        assertEquals("platform", clusteringUDT.field(2).name());
+        assertEquals(bridge.text(), clusteringUDT.field(2).type());
+        assertEquals("time_range", clusteringUDT.field(3).name());
+        assertEquals(bridge.text(), clusteringUDT.field(3).type());
+
+        CqlField.CqlUdt fieldsUDT = (CqlField.CqlUdt) ((CqlField.CqlFrozen) fields.get(2).type()).inner();
+        assertEquals("first_last_seen_fields_v1", fieldsUDT.name());
+        assertEquals("firstSeen", fieldsUDT.field(0).name());
+        assertEquals("field_with_timestamp", ((CqlField.CqlFrozen) fieldsUDT.field(0).type()).inner().name());
+        assertEquals("lastSeen", fieldsUDT.field(1).name());
+        assertEquals("field_with_timestamp", ((CqlField.CqlFrozen) fieldsUDT.field(1).type()).inner().name());
+        assertEquals("firstTransaction", fieldsUDT.field(2).name());
+        assertEquals("field_with_timestamp", ((CqlField.CqlFrozen) fieldsUDT.field(2).type()).inner().name());
+        assertEquals("lastTransaction", fieldsUDT.field(3).name());
+        assertEquals("field_with_timestamp", ((CqlField.CqlFrozen) fieldsUDT.field(3).type()).inner().name());
+        assertEquals("firstListening", fieldsUDT.field(4).name());
+        assertEquals("field_with_timestamp", ((CqlField.CqlFrozen) fieldsUDT.field(4).type()).inner().name());
+        assertEquals("lastListening", fieldsUDT.field(5).name());
+        assertEquals("field_with_timestamp", ((CqlField.CqlFrozen) fieldsUDT.field(5).type()).inner().name());
+        assertEquals("firstReading", fieldsUDT.field(6).name());
+        assertEquals("field_with_timestamp", ((CqlField.CqlFrozen) fieldsUDT.field(6).type()).inner().name());
+        assertEquals("lastReading", fieldsUDT.field(7).name());
+        assertEquals("field_with_timestamp", ((CqlField.CqlFrozen) fieldsUDT.field(7).type()).inner().name());
+        assertEquals("outputEvent", fieldsUDT.field(8).name());
+        assertEquals(bridge.text(), fieldsUDT.field(8).type());
+        assertEquals("eventHistory", fieldsUDT.field(9).name());
+        assertEquals(bridge.bigint(),
+                     ((CqlField.CqlMap) ((CqlField.CqlFrozen) fieldsUDT.field(9).type()).inner()).keyType());
+        assertEquals(CqlField.CqlType.InternalType.Frozen,
+                     ((CqlField.CqlMap) ((CqlField.CqlFrozen) fieldsUDT.field(9).type()).inner()).valueType().internalType());
+    }
+
+    @Test
+    public void testNestedUDTs()
+    {
+        ReplicationFactor replicationFactor = new ReplicationFactor(
+                ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy, ImmutableMap.of("DC1", 3, "DC2", 3));
+        String keyspace = "nested_udt_schema";
+        CqlTable table = bridge.buildSchema("CREATE TABLE " + keyspace + ".udt_test (\n"
+                                          + "    a uuid,\n"
+                                          + "    b bigint,\n"
+                                          + "    c frozen<a_udt>,\n"
+                                          + "    PRIMARY KEY(a));", keyspace, replicationFactor, Partitioner.Murmur3Partitioner,
+                                            ImmutableSet.of("CREATE TYPE " + keyspace + ".a_udt (col1 bigint, col2 text, col3 frozen<map<uuid, b_udt>>);",
+                                                           "CREATE TYPE " + keyspace + ".b_udt (col1 timeuuid, col2 text, col3 frozen<set<c_udt>>);",
+                                                           "CREATE TYPE " + keyspace + ".c_udt (col1 float, col2 uuid, col3 int);"));
+        assertEquals(3, table.udts().size());
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/reader/TombstoneWriterTests.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/reader/TombstoneWriterTests.java
new file mode 100644
index 0000000..b200a30
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/reader/TombstoneWriterTests.java
@@ -0,0 +1,204 @@
+/*
+ * 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.cassandra.spark.reader;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.nio.file.Path;
+
+import org.junit.Test;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.cassandra.spark.TestUtils;
+import org.apache.cassandra.spark.data.FileType;
+import org.apache.cassandra.spark.utils.test.TestSchema;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.quicktheories.QuickTheory.qt;
+
+/**
+ * Test we can write out partition, row and range tombstones to SSTables using the SSTableTombstoneWriter
+ */
+public class TombstoneWriterTests
+{
+    private static final int NUM_ROWS = 50;
+    private static final ObjectMapper MAPPER = new ObjectMapper();
+
+    @Test
+    public void testPartitionTombstone()
+    {
+        qt().forAll(TestUtils.tombstoneVersions())
+            .checkAssert(version -> TestUtils.runTest(version, (partitioner, directory, bridge) -> {
+                // Write tombstone SSTable
+                TestSchema schema = TestSchema.basicBuilder(bridge)
+                                              .withDeleteFields("a =")
+                                              .build();
+                schema.writeTombstoneSSTable(directory, bridge, partitioner, writer -> {
+                    for (int index = 0; index < NUM_ROWS; index++)
+                    {
+                        writer.write(index);
+                    }
+                });
+
+                // Convert SSTable to JSON
+                Path dataDbFile = TestUtils.getFirstFileType(directory, FileType.DATA);
+                ByteArrayOutputStream out = new ByteArrayOutputStream();
+                bridge.sstableToJson(dataDbFile, out);
+                JsonNode node;
+                try
+                {
+                    node = MAPPER.readTree(out.toByteArray());
+                }
+                catch (IOException exception)
+                {
+                    throw new RuntimeException(exception);
+                }
+
+                // Verify SSTable contains partition tombstones
+                assertEquals(NUM_ROWS, node.size());
+                for (int index = 0; index < NUM_ROWS; index++)
+                {
+                    JsonNode partition = node.get(index).get("partition");
+                    int key = partition.get("key").get(0).asInt();
+                    assertTrue(0 <= key && key < NUM_ROWS);
+                    assertTrue(node.get(index).has("rows"));
+                    assertTrue(partition.has("deletion_info"));
+                    assertTrue(partition.get("deletion_info").has("marked_deleted"));
+                    assertTrue(partition.get("deletion_info").has("local_delete_time"));
+                }
+            }));
+    }
+
+    @Test
+    public void testRowTombstone()
+    {
+        qt().forAll(TestUtils.tombstoneVersions())
+            .checkAssert(version -> TestUtils.runTest(version, (partitioner, directory, bridge) -> {
+                // Write tombstone SSTable
+                TestSchema schema = TestSchema.basicBuilder(bridge)
+                                              .withDeleteFields("a =", "b =")
+                                              .build();
+                schema.writeTombstoneSSTable(directory, bridge, partitioner, writer -> {
+                    for (int index = 0; index < NUM_ROWS; index++)
+                    {
+                        writer.write(index, index);
+                    }
+                });
+
+                // Convert SSTable to JSON
+                Path dataDbFile = TestUtils.getFirstFileType(directory, FileType.DATA);
+                ByteArrayOutputStream out = new ByteArrayOutputStream();
+                bridge.sstableToJson(dataDbFile, out);
+                JsonNode node;
+                try
+                {
+                    node = MAPPER.readTree(out.toByteArray());
+                }
+                catch (IOException exception)
+                {
+                    throw new RuntimeException(exception);
+                }
+
+                // Verify SSTable contains row tombstones
+                assertEquals(NUM_ROWS, node.size());
+                for (int index = 0; index < NUM_ROWS; index++)
+                {
+                    JsonNode partition = node.get(index).get("partition");
+                    int key = partition.get("key").get(0).asInt();
+                    assertTrue(0 <= key && key < NUM_ROWS);
+                    assertFalse(partition.has("deletion_info"));
+
+                    assertTrue(node.get(index).has("rows"));
+                    JsonNode row = node.get(index).get("rows").get(0);
+                    assertEquals("row", row.get("type").asText());
+                    assertEquals(key, row.get("clustering").get(0).asInt());
+                    assertTrue(row.has("deletion_info"));
+                    assertTrue(row.get("deletion_info").has("marked_deleted"));
+                    assertTrue(row.get("deletion_info").has("local_delete_time"));
+                }
+            }));
+    }
+
+    @Test
+    public void testRangeTombstone()
+    {
+        qt().forAll(TestUtils.tombstoneVersions())
+            .checkAssert(version -> TestUtils.runTest(version, (partitioner, directory, bridge) -> {
+                // Write tombstone SSTable
+                TestSchema schema = TestSchema.basicBuilder(bridge)
+                                              .withDeleteFields("a =", "b >=", "b <")
+                                              .build();
+                schema.writeTombstoneSSTable(directory, bridge, partitioner, writer -> {
+                    for (int index = 0; index < NUM_ROWS; index++)
+                    {
+                        writer.write(index, 50, 999);
+                    }
+                });
+
+                // Convert SSTable to JSON
+                Path dataDbFile = TestUtils.getFirstFileType(directory, FileType.DATA);
+                ByteArrayOutputStream out = new ByteArrayOutputStream();
+                bridge.sstableToJson(dataDbFile, out);
+                JsonNode node;
+                try
+                {
+                    node = MAPPER.readTree(out.toByteArray());
+                }
+                catch (IOException exception)
+                {
+                    throw new RuntimeException(exception);
+                }
+
+                // Verify SSTable contains range tombstones
+                assertEquals(NUM_ROWS, node.size());
+                for (int index = 0; index < NUM_ROWS; index++)
+                {
+                    JsonNode partition = node.get(index).get("partition");
+                    int key = partition.get("key").get(0).asInt();
+                    assertTrue(0 <= key && key < NUM_ROWS);
+                    assertFalse(partition.has("deletion_info"));
+
+                    assertTrue(node.get(index).has("rows"));
+                    assertEquals(2, node.get(index).get("rows").size());
+
+                    JsonNode row1 = node.get(index).get("rows").get(0);
+                    assertEquals("range_tombstone_bound", row1.get("type").asText());
+                    JsonNode start = row1.get("start");
+                    assertEquals("inclusive", start.get("type").asText());
+                    assertEquals(50, start.get("clustering").get(0).asInt());
+                    assertTrue(start.has("deletion_info"));
+                    assertTrue(start.get("deletion_info").has("marked_deleted"));
+                    assertTrue(start.get("deletion_info").has("local_delete_time"));
+
+                    JsonNode row2 = node.get(index).get("rows").get(1);
+                    assertEquals("range_tombstone_bound", row2.get("type").asText());
+                    JsonNode end = row2.get("end");
+                    assertEquals("exclusive", end.get("type").asText());
+                    assertEquals(999, end.get("clustering").get(0).asInt());
+                    assertTrue(end.has("deletion_info"));
+                    assertTrue(end.get("deletion_info").has("marked_deleted"));
+                    assertTrue(end.get("deletion_info").has("local_delete_time"));
+                }
+            }));
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/reader/common/CompressionInputStreamTests.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/reader/common/CompressionInputStreamTests.java
new file mode 100644
index 0000000..91e9adf
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/reader/common/CompressionInputStreamTests.java
@@ -0,0 +1,125 @@
+/*
+ * 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.cassandra.spark.reader.common;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.UUID;
+
+import org.junit.Test;
+
+import org.apache.cassandra.spark.stats.Stats;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class CompressionInputStreamTests
+{
+    @Test
+    public void testRawInputStream() throws IOException
+    {
+        String filename = UUID.randomUUID().toString();
+        Path path = Files.createTempFile(filename, "tmp");
+        try (DataOutputStream dos = new DataOutputStream(new BufferedOutputStream(new FileOutputStream(path.toFile()))))
+        {
+            dos.writeUTF(filename);
+            int numWrites = 1000;
+            dos.writeInt(numWrites);
+            for (int write = 0; write < numWrites; write++)
+            {
+                dos.writeInt(write);
+            }
+            for (int write = 0; write < numWrites; write++)
+            {
+                dos.writeLong(write);
+            }
+            dos.writeBoolean(false);
+            dos.writeBoolean(true);
+        }
+
+        byte[] buffer = new byte[1024];
+        try (DataInputStream dis = new DataInputStream(new RawInputStream(new DataInputStream(
+                new BufferedInputStream(Files.newInputStream(path))), buffer, Stats.DoNothingStats.INSTANCE)))
+        {
+            assertEquals(filename, dis.readUTF());
+            int numReads = dis.readInt();
+            for (int read = 0; read < numReads; read++)
+            {
+                assertEquals(read, dis.readInt());
+            }
+            for (int read = 0; read < numReads; read++)
+            {
+                assertEquals(read, dis.readLong());
+            }
+            assertFalse(dis.readBoolean());
+            assertTrue(dis.readBoolean());
+        }
+    }
+
+    @Test(expected = IndexOutOfBoundsException.class)
+    public void testBigLongArrayIllegalSize()
+    {
+        new BigLongArray(-1);
+    }
+
+    @Test(expected = IndexOutOfBoundsException.class)
+    public void testBigLongArrayEmpty()
+    {
+        BigLongArray array = new BigLongArray(0);
+        array.set(0, 0L);
+    }
+
+    @Test(expected = IndexOutOfBoundsException.class)
+    public void testBigLongArrayOutOfRange()
+    {
+        BigLongArray array = new BigLongArray(500);
+        array.set(501, 999L);
+    }
+
+    @Test
+    public void testBigLongArrayUnary()
+    {
+        BigLongArray array = new BigLongArray(1);
+        array.set(0, 999L);
+        assertEquals(999L, array.get(0));
+    }
+
+    @Test
+    public void testBigLongArray()
+    {
+        int size = BigLongArray.DEFAULT_PAGE_SIZE * 37;
+        BigLongArray array = new BigLongArray(size);
+        for (int index = 0; index < size; index++)
+        {
+            array.set(index, index * 5L);
+        }
+        for (int index = 0; index < size; index++)
+        {
+            assertEquals(index * 5L, array.get(index));
+        }
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/utils/BuildInfoTest.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/utils/BuildInfoTest.java
new file mode 100644
index 0000000..836989b
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/utils/BuildInfoTest.java
@@ -0,0 +1,78 @@
+/*
+ * 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.cassandra.spark.utils;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+public class BuildInfoTest
+{
+    @Test
+    public void testReaderUserAgent()
+    {
+        assertTrue(BuildInfo.READER_USER_AGENT.endsWith(" reader"));
+        assertNotEquals("unknown", BuildInfo.getBuildVersion());
+    }
+
+    @Test
+    public void testWriterUserAgent()
+    {
+        assertTrue(BuildInfo.WRITER_USER_AGENT.endsWith(" writer"));
+        assertNotEquals("unknown", BuildInfo.getBuildVersion());
+    }
+
+    @Test
+    public void testJavaVersionReturnsAValue()
+    {
+        assertNotNull(BuildInfo.javaSpecificationVersion());
+    }
+
+    @Test
+    public void isAtLeastJava11()
+    {
+        assertFalse(BuildInfo.isAtLeastJava11(null));
+        assertFalse(BuildInfo.isAtLeastJava11("0.9"));
+        assertFalse(BuildInfo.isAtLeastJava11("1.1"));
+        assertFalse(BuildInfo.isAtLeastJava11("1.2"));
+        assertFalse(BuildInfo.isAtLeastJava11("1.3"));
+        assertFalse(BuildInfo.isAtLeastJava11("1.4"));
+        assertFalse(BuildInfo.isAtLeastJava11("1.5"));
+        assertFalse(BuildInfo.isAtLeastJava11("1.6"));
+        assertFalse(BuildInfo.isAtLeastJava11("1.7"));
+        assertFalse(BuildInfo.isAtLeastJava11("1.8"));
+        assertFalse(BuildInfo.isAtLeastJava11("9"));
+        assertFalse(BuildInfo.isAtLeastJava11("10"));
+        assertTrue(BuildInfo.isAtLeastJava11("11"));
+        assertTrue(BuildInfo.isAtLeastJava11("12"));
+        assertTrue(BuildInfo.isAtLeastJava11("13"));
+        assertTrue(BuildInfo.isAtLeastJava11("14"));
+        assertTrue(BuildInfo.isAtLeastJava11("15"));
+        assertTrue(BuildInfo.isAtLeastJava11("16"));
+        assertTrue(BuildInfo.isAtLeastJava11("17"));
+        assertTrue(BuildInfo.isAtLeastJava11("18"));
+        assertTrue(BuildInfo.isAtLeastJava11("19"));
+        assertTrue(BuildInfo.isAtLeastJava11("20"));
+        assertTrue(BuildInfo.isAtLeastJava11("21"));
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/utils/ByteBufferUtilsTests.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/utils/ByteBufferUtilsTests.java
new file mode 100644
index 0000000..9f81670
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/utils/ByteBufferUtilsTests.java
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.cassandra.spark.utils;
+
+import java.io.ByteArrayInputStream;
+import java.io.EOFException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+
+import com.google.common.io.ByteArrayDataInput;
+import com.google.common.io.ByteStreams;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+@SuppressWarnings("UnstableApiUsage")
+public class ByteBufferUtilsTests
+{
+    @Test
+    public void testSkipBytesFully() throws IOException
+    {
+        testSkipBytesFully("abc".getBytes(StandardCharsets.UTF_8));
+        testSkipBytesFully("abcdefghijklm".getBytes(StandardCharsets.UTF_8));
+    }
+
+    @Test
+    public void testReadRemainingBytes() throws IOException
+    {
+        testReadRemainingBytes("");
+        testReadRemainingBytes("abc");
+        testReadRemainingBytes("abcdefghijklm");
+    }
+
+    @Test
+    public void testGetArray()
+    {
+        testGetArray("");
+        testGetArray("abc");
+        testGetArray("abcdefghijklm");
+    }
+
+    @Test
+    public void testHexString()
+    {
+        // Casts to (ByteBuffer) required when compiling with Java 8
+        assertEquals("00000000000001F4", ByteBufferUtils.toHexString((ByteBuffer) ByteBuffer.allocate(8).putLong(500L).flip()));
+        assertEquals("616263", ByteBufferUtils.toHexString(ByteBuffer.wrap(new byte[]{'a', 'b', 'c'})));
+        assertEquals("000000000588C164", ByteBufferUtils.toHexString((ByteBuffer) ByteBuffer.allocate(8).putLong(92848484L).asReadOnlyBuffer().flip()));
+        assertEquals("null", ByteBufferUtils.toHexString(null));
+
+        assertEquals("616263", ByteBufferUtils.toHexString(new byte[]{'a', 'b', 'c'}, 0, 3));
+        assertEquals("63", ByteBufferUtils.toHexString(new byte[]{'a', 'b', 'c'}, 2, 1));
+    }
+
+    private static void testGetArray(String str)
+    {
+        assertEquals(str, new String(ByteBufferUtils.getArray(ByteBuffer.wrap(str.getBytes())), StandardCharsets.UTF_8));
+    }
+
+    private static void testReadRemainingBytes(String str) throws IOException
+    {
+        assertEquals(str, new String(ByteBufferUtils.readRemainingBytes(new ByteArrayInputStream(str.getBytes()), str.length()), StandardCharsets.UTF_8));
+    }
+
+    private static void testSkipBytesFully(byte[] bytes) throws IOException
+    {
+        int length = bytes.length;
+        ByteArrayDataInput in = ByteStreams.newDataInput(bytes, 0);
+        ByteBufferUtils.skipBytesFully(in, 1);
+        ByteBufferUtils.skipBytesFully(in, length - 2);
+        assertEquals(bytes[length - 1], in.readByte());
+        try
+        {
+            ByteBufferUtils.skipBytesFully(in, 1);
+            fail("EOFException should have been thrown");
+        }
+        catch (EOFException ignore)
+        {
+        }
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/utils/CqlUtilsTest.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/utils/CqlUtilsTest.java
new file mode 100644
index 0000000..8b4ffaa
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/utils/CqlUtilsTest.java
@@ -0,0 +1,678 @@
+/*
+ * 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.cassandra.spark.utils;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URISyntaxException;
+import java.nio.charset.StandardCharsets;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.commons.io.FileUtils;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.bridge.CassandraVersion;
+import org.apache.cassandra.spark.data.CqlTable;
+import org.apache.cassandra.spark.data.ReplicationFactor;
+import org.apache.cassandra.spark.data.VersionRunner;
+import org.apache.cassandra.spark.data.partitioner.Partitioner;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * Unit tests for {@link CqlUtils}
+ */
+public class CqlUtilsTest extends VersionRunner
+{
+    static String fullSchemaSample;
+
+    public CqlUtilsTest(CassandraVersion version)
+    {
+        super(version);
+    }
+
+    @BeforeClass
+    public static void setup() throws URISyntaxException, IOException
+    {
+        fullSchemaSample = loadFullSchemaSample();
+    }
+
+    @Test
+    public void textExtractIndexCount()
+    {
+        int indexCount = CqlUtils.extractIndexCount(fullSchemaSample, "cycling", "rank_by_year_and_name");
+        assertEquals(3, indexCount);
+    }
+
+    @Test
+    public void testExtractKeyspace()
+    {
+        String keyspaceSchema = CqlUtils.extractKeyspaceSchema(fullSchemaSample, "keyspace");
+        String tagEntityRelationV4KeyspaceSchema = CqlUtils.extractKeyspaceSchema(fullSchemaSample, "quoted_keyspace");
+        String systemDistributedKeyspaceSchema = CqlUtils.extractKeyspaceSchema(fullSchemaSample, "system_distributed");
+        String systemSchemaKeyspaceSchema = CqlUtils.extractKeyspaceSchema(fullSchemaSample, "system_schema");
+        assertEquals("CREATE KEYSPACE keyspace "
+                   + "WITH REPLICATION = { 'class' : 'org.apache.cassandra.locator.NetworkTopologyStrategy', "
+                   +                      "'datacenter1': '4', "
+                   +                      "'datacenter2': '3' } AND DURABLE_WRITES = true;", keyspaceSchema);
+        assertEquals("CREATE KEYSPACE \"quoted_keyspace\" "
+                   + "WITH REPLICATION = { 'class' : 'org.apache.cassandra.locator.NetworkTopologyStrategy', "
+                   +                      "'datacenter1': '3', "
+                   +                      "'datacenter2': '3' } AND DURABLE_WRITES = true;", tagEntityRelationV4KeyspaceSchema);
+        assertEquals("CREATE KEYSPACE system_distributed "
+                   + "WITH REPLICATION = { 'class' : 'org.apache.cassandra.locator.SimpleStrategy', "
+                   +                      "'replication_factor': '3' } AND DURABLE_WRITES = true;", systemDistributedKeyspaceSchema);
+        assertEquals("CREATE KEYSPACE system_schema "
+                   + "WITH REPLICATION = { 'class' : 'org.apache.cassandra.locator.LocalStrategy' } AND DURABLE_WRITES = true;", systemSchemaKeyspaceSchema);
+    }
+
+    @Test
+    public void testExtractKeyspaceNames()
+    {
+        Set<String> keyspaceNames = CqlUtils.extractKeyspaceNames(fullSchemaSample);
+        assertEquals(3, keyspaceNames.size());
+        Map<String, ReplicationFactor> rfMap = keyspaceNames
+                .stream()
+                .collect(Collectors.toMap(Function.identity(),
+                                          keyspace -> CqlUtils.extractReplicationFactor(fullSchemaSample, keyspace)));
+        assertTrue(rfMap.containsKey("keyspace"));
+        assertTrue(rfMap.containsKey("quoted_keyspace"));
+        assertEquals(4, rfMap.get("keyspace").getOptions().get("datacenter1").intValue());
+        assertEquals(3, rfMap.get("keyspace").getOptions().get("datacenter2").intValue());
+        assertEquals(3, rfMap.get("quoted_keyspace").getOptions().get("datacenter1").intValue());
+        assertEquals(3, rfMap.get("quoted_keyspace").getOptions().get("datacenter2").intValue());
+    }
+
+    @Test
+    public void testExtractReplicationFactor()
+    {
+        ReplicationFactor keyspaceRf = CqlUtils.extractReplicationFactor(fullSchemaSample, "keyspace");
+        assertNotNull(keyspaceRf);
+        assertEquals(ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy, keyspaceRf.getReplicationStrategy());
+        assertEquals(7, keyspaceRf.getTotalReplicationFactor().intValue());
+        assertEquals(ImmutableMap.of("datacenter1", 4, "datacenter2", 3), keyspaceRf.getOptions());
+
+        ReplicationFactor tagEntityRelationV4Rf = CqlUtils.extractReplicationFactor(fullSchemaSample, "quoted_keyspace");
+        assertNotNull(tagEntityRelationV4Rf);
+        assertEquals(ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy, tagEntityRelationV4Rf.getReplicationStrategy());
+        assertEquals(6, tagEntityRelationV4Rf.getTotalReplicationFactor().intValue());
+        assertEquals(ImmutableMap.of("datacenter1", 3, "datacenter2", 3), tagEntityRelationV4Rf.getOptions());
+
+        ReplicationFactor systemDistributedRf = CqlUtils.extractReplicationFactor(fullSchemaSample, "system_distributed");
+        assertNotNull(systemDistributedRf);
+        assertEquals(ReplicationFactor.ReplicationStrategy.SimpleStrategy, systemDistributedRf.getReplicationStrategy());
+        assertEquals(3, systemDistributedRf.getTotalReplicationFactor().intValue());
+        assertEquals(ImmutableMap.of("replication_factor", 3), systemDistributedRf.getOptions());
+
+        ReplicationFactor systemSchemaRf = CqlUtils.extractReplicationFactor(fullSchemaSample, "system_schema");
+        assertNotNull(systemSchemaRf);
+        assertEquals(ReplicationFactor.ReplicationStrategy.LocalStrategy, systemSchemaRf.getReplicationStrategy());
+        assertEquals(0, systemSchemaRf.getTotalReplicationFactor().intValue());
+        assertEquals(ImmutableMap.of(), systemSchemaRf.getOptions());
+    }
+
+    @Test
+    public void testEscapedColumnNames()
+    {
+        String cleaned = CqlUtils.extractTableSchema(fullSchemaSample, "cycling", "rank_by_year_and_name_quoted_columns");
+        assertEquals("CREATE TABLE cycling.rank_by_year_and_name_quoted_columns("
+                   + "    race_year      int,"
+                   + "    \"RACE_NAME\"    text,"
+                   + "    rank           int,"
+                   + "    \"cyclist_Name\" text,"
+                   + "    PRIMARY KEY ((race_year, \"RACE_NAME\"), rank)) "
+                   + "WITH CLUSTERING ORDER BY (rank ASC)"
+                   + " AND min_index_interval = 128"
+                   + " AND max_index_interval = 2048"
+                   + " AND cdc = false;", cleaned);
+    }
+
+    @Test
+    public void testExtractTableSchemaCase1()
+    {
+        String schemaStr = "CREATE TABLE keyspace.table ("
+                         + "key blob, "
+                         + "column1 text, "
+                         + "\"C0\" counter static, "
+                         + "\"C1\" counter static, "
+                         + "\"C2\" counter static, "
+                         + "\"C3\" counter static, "
+                         + "\"C4\" counter static, "
+                         + "value counter, "
+                         + "PRIMARY KEY (key, column1) "
+                         + ") WITH COMPACT STORAGE "
+                         + "AND CLUSTERING ORDER BY (column1 ASC) "
+                         + "AND cdc = false "
+                         + "AND bloom_filter_fp_chance = 0.1 "
+                         + "AND caching = {'keys': 'ALL', 'rows_per_partition': 'NONE'} "
+                         + "AND comment = '' "
+                         + "AND compaction = {'class': 'org.apache.cassandra.db.compaction.LeveledCompactionStrategy'} "
+                         + "AND compression = {'chunk_length_in_kb': '64', "
+                         +                    "'class': 'org.apache.cassandra.io.compress.DeflateCompressor'} "
+                         + "AND crc_check_chance = 1.0 "
+                         + "AND dclocal_read_repair_chance = 0.1 "
+                         + "AND default_time_to_live = 0 "
+                         + "AND gc_grace_seconds = 864000 "
+                         + "AND max_index_interval = 2048 "
+                         + "AND memtable_flush_period_in_ms = 0 "
+                         + "AND min_index_interval = 128  "
+                         + "AND read_repair_chance = 0.0 "
+                         + "AND speculative_retry = '99p';";
+        String expectedCreateStmt = "CREATE TABLE keyspace.table ("
+                                  + "key blob, "
+                                  + "column1 text, "
+                                  + "\"C0\" counter static, "
+                                  + "\"C1\" counter static, "
+                                  + "\"C2\" counter static, "
+                                  + "\"C3\" counter static, "
+                                  + "\"C4\" counter static, "
+                                  + "value counter, "
+                                  + "PRIMARY KEY (key, column1) ) WITH CLUSTERING ORDER BY (column1 ASC) "
+                                  + "AND cdc = false AND max_index_interval = 2048 AND min_index_interval = 128;";
+        String actualCreateStmt = CqlUtils.extractTableSchema(schemaStr, "keyspace", "table");
+        assertEquals(expectedCreateStmt, actualCreateStmt);
+    }
+
+    @Test
+    public void testFailsWithUnbalancedParenthesis()
+    {
+        String schemaStr = "CREATE TABLE keyspace.table (key blob, c0 text, c1 text, PRIMARY KEY (key);";
+
+        try
+        {
+            CqlUtils.extractTableSchema(schemaStr, "keyspace", "table");
+            fail("Expected RuntimeException when parentheses are unbalanced");
+        }
+        catch (RuntimeException exception)
+        {
+            assertEquals("Found unbalanced parentheses in table schema "
+                       + "CREATE TABLE keyspace.table (key blob, c0 text, c1 text, PRIMARY KEY (key);",
+                         exception.getMessage());
+        }
+    }
+
+    @Test
+    public void testExtractTableSchemaCase2()
+    {
+        String schemaStr = "CREATE TABLE keyspace.table ("
+                         + "key blob, "
+                         + "column1 text, "
+                         + "\"C0\" blob, "
+                         + "\"C1\" blob, "
+                         + "\"C2\" blob, "
+                         + "\"C4\" blob, "
+                         + "value counter, "
+                         + "PRIMARY KEY (key, column1) "
+                         + ") WITH bloom_filter_fp_chance = 0.1 "
+                         + "AND cdc = true "
+                         + "AND caching = {'keys': 'ALL', 'rows_per_partition': 'NONE'} "
+                         + "AND comment = '' "
+                         + "AND compaction = {'class': 'org.apache.cassandra.db.compaction.LeveledCompactionStrategy'} "
+                         + "AND compression = {'chunk_length_in_kb': '64', "
+                         +                    "'class': 'org.apache.cassandra.io.compress.DeflateCompressor'} "
+                         + "AND crc_check_chance = 1.0 "
+                         + "AND dclocal_read_repair_chance = 0.1 "
+                         + "AND default_time_to_live = 0 "
+                         + "AND gc_grace_seconds = 864000 "
+                         + "AND max_index_interval = 2048 "
+                         + "AND memtable_flush_period_in_ms = 0 "
+                         + "AND min_index_interval = 128 "
+                         + "AND read_repair_chance = 0.0 "
+                         + "AND speculative_retry = '99p';";
+        String expectedCreateStmt = "CREATE TABLE keyspace.table ("
+                                  + "key blob, "
+                                  + "column1 text, "
+                                  + "\"C0\" blob, "
+                                  + "\"C1\" blob, "
+                                  + "\"C2\" blob, "
+                                  + "\"C4\" blob, "
+                                  + "value counter, "
+                                  + "PRIMARY KEY (key, column1) ) WITH "
+                                  + "cdc = true AND max_index_interval = 2048 AND min_index_interval = 128;";
+        String actualCreateStmt = CqlUtils.extractTableSchema(schemaStr, "keyspace", "table");
+        assert expectedCreateStmt.equals(actualCreateStmt);
+    }
+
+    @Test
+    public void testEscapedTableName()
+    {
+        String schemaStr = "CREATE TABLE ks.\\\"tb\\\" (\\n"
+                         + "\\\"key\\\" text,\\n"
+                         + " \\\"id1\\\" text,\\n"
+                         + " \\\"id2\\\" text,\\n"
+                         + " \\\"id3\\\" text,\\n"
+                         + " created timestamp,\\n"
+                         + " id4 uuid,\\n metadata blob,\\n"
+                         + " PRIMARY KEY ((\\\"key\\\", \\\"id1\\\"), \\\"id2\\\", \\\"id3\\\")\\n) "
+                         + "WITH CLUSTERING ORDER BY (\\\"id2\\\" DESC, \\\"id3\\\" ASC)\\n"
+                         + "    AND read_repair_chance = 0.0\\n"
+                         + "    AND dclocal_read_repair_chance = 0.1\\n"
+                         + "    AND gc_grace_seconds = 864000\\n"
+                         + "    AND bloom_filter_fp_chance = 0.1\\n"
+                         + "    AND caching = { 'keys' : 'ALL', 'rows_per_partition' : 'NONE' }\\n"
+                         + "    AND comment = ''\\n"
+                         + "    AND compaction = { 'class' : 'org.apache.cassandra.db.compaction.LeveledCompactionStrategy', "
+                         +                        "'max_threshold' : 32, "
+                         +                        "'min_threshold' : 4 }\\n"
+                         + "    AND compression = { 'chunk_length_in_kb' : 64, "
+                         +                         "'class' : 'org.apache.cassandra.io.compress.LZ4Compressor' }\\n"
+                         + "    AND default_time_to_live = 0\\n"
+                         + "    AND speculative_retry = '99p'\\n"
+                         + "    AND min_index_interval = 128\\n"
+                         + "    AND max_index_interval = 2048\\n"
+                         + "    AND crc_check_chance = 1.0\\n"
+                         + "    AND memtable_flush_period_in_ms = 0;";
+        String expectedCreateStmt = "CREATE TABLE ks.tb ("
+                                  + "\"key\" text, "
+                                  + "\"id1\" text, "
+                                  + "\"id2\" text, "
+                                  + "\"id3\" text, "
+                                  + "created timestamp, "
+                                  + "id4 uuid, "
+                                  + "metadata blob, "
+                                  + "PRIMARY KEY ((\"key\", \"id1\"), \"id2\", \"id3\")) "
+                                  + "WITH CLUSTERING ORDER BY (\"id2\" DESC, \"id3\" ASC)"
+                                  + " AND min_index_interval = 128"
+                                  + " AND max_index_interval = 2048;";
+        String actualCreateStmt = CqlUtils.extractTableSchema(schemaStr, "ks", "tb");
+        assertEquals(expectedCreateStmt, actualCreateStmt);
+        CqlTable table = bridge.buildSchema(actualCreateStmt,
+                                            "ks",
+                                            new ReplicationFactor(ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy,
+                                                                  ImmutableMap.of("datacenter1", 3)),
+                                            Partitioner.Murmur3Partitioner,
+                                            Collections.emptySet(),
+                                            null, 0);
+        assertEquals("ks", table.keyspace());
+        assertEquals("tb", table.table());
+        assertEquals("key", table.getField("key").name());
+        assertEquals("id1", table.getField("id1").name());
+    }
+
+    @Test
+    public void testCleanTableName()
+    {
+        assertEquals("test", CqlUtils.cleanTableName("test"));
+        assertEquals("test", CqlUtils.cleanTableName("\"test\""));
+        assertEquals("test", CqlUtils.cleanTableName("\"\"test\"\""));
+    }
+
+    @Test
+    public void testExtractTableSchemaCase3()
+    {
+        String schemaStr = "CREATE TABLE keyspace.table ("
+                         + "key blob, "
+                         + "column1 text, "
+                         + "\"C0\" blob, "
+                         + "\"C1\" blob, "
+                         + "\"C2\" blob, "
+                         + "\"C4\" blob, "
+                         + "value counter, "
+                         + "PRIMARY KEY ((key, column1), value) "
+                         + ") WITH bloom_filter_fp_chance = 0.1 "
+                         + "AND caching = {'keys': 'ALL', 'rows_per_partition': 'NONE'} "
+                         + "AND comment = '' "
+                         + "AND compaction = {'class': 'org.apache.cassandra.db.compaction.LeveledCompactionStrategy'} "
+                         + "AND compression = {'chunk_length_in_kb': '64', "
+                         +                    "'class': 'org.apache.cassandra.io.compress.DeflateCompressor'} "
+                         + "AND crc_check_chance = 1.0 "
+                         + "AND dclocal_read_repair_chance = 0.1 "
+                         + "AND default_time_to_live = 0 "
+                         + "AND gc_grace_seconds = 864000 "
+                         + "AND max_index_interval = 2048 "
+                         + "AND memtable_flush_period_in_ms = 0 "
+                         + "AND min_index_interval = 128 "
+                         + "AND read_repair_chance = 0.0 "
+                         + "AND speculative_retry = '99p';";
+        String expectedCreateStmt = "CREATE TABLE keyspace.table ("
+                                  + "key blob, "
+                                  + "column1 text, "
+                                  + "\"C0\" blob, "
+                                  + "\"C1\" blob, "
+                                  + "\"C2\" blob, "
+                                  + "\"C4\" blob, "
+                                  + "value counter, "
+                                  + "PRIMARY KEY ((key, column1), value) ) "
+                                  + "WITH max_index_interval = 2048 AND min_index_interval = 128;";
+        String actualCreateStmt = CqlUtils.extractTableSchema(schemaStr, "keyspace", "table");
+        assert expectedCreateStmt.equals(actualCreateStmt);
+    }
+
+    @Test
+    public void testBasicExtractUDTs()
+    {
+        String schemaStr = "CREATE TYPE udt_keyspace.test_idt1 (a text, b bigint, c int, d float);\n"
+                         + "CREATE TYPE udt_keyspace.test_idt2 (x boolean, y timestamp, z timeuuid);";
+        Set<String> udts = CqlUtils.extractUdts(schemaStr, "udt_keyspace");
+        assertEquals(2, udts.size());
+        assertTrue(udts.contains("CREATE TYPE udt_keyspace.test_idt1 (a text, b bigint, c int, d float);"));
+        assertTrue(udts.contains("CREATE TYPE udt_keyspace.test_idt2 (x boolean, y timestamp, z timeuuid);"));
+    }
+
+    @Test
+    public void testExtractUDTs()
+    {
+        String schema = "\"CREATE TYPE some_keyspace.udt123 (\\n\" +\n"
+                      + "                              \"    x uuid,\\n\" +\n"
+                      + "                              \"    y text,\\n\" +\n"
+                      + "                              \"    z uuid\\n\" +\n"
+                      + "                              \");\\n\" +"
+                      + "\"CREATE KEYSPACE udt_keyspace "
+                      + "WITH REPLICATION = { 'class' : 'org.apache.cassandra.locator.NetworkTopologyStrategy', "
+                      +                      "'datacenter1': '3' } AND DURABLE_WRITES = true;\n"
+                      + "CREATE TYPE udt_keyspace.type_with_time_zone (\n"
+                      + "    time bigint,\n"
+                      + "    \\\"timezoneOffsetMinutes\\\" int\n"
+                      + ");\n"
+                      + "CREATE TYPE udt_keyspace.type_1 (\n"
+                      + "    \\\"x\\\" text,\n"
+                      + "    \\\"y\\\" text,\n"
+                      + "    z text,\n"
+                      + "    \\\"a\\\" boolean\n"
+                      + ");\n"
+                      + "CREATE TYPE udt_keyspace.field_with_timestamp (\n"
+                      + "    field text,\n"
+                      + "    \\\"timeWithZone\\\" frozen<udt_keyspace.type_with_time_zone>\n"
+                      + ");\n"
+                      + "CREATE TYPE udt_keyspace.type_with_frozen_fields (\n"
+                      + "    \\\"f1\\\" frozen<udt_keyspace.field_with_timestamp>,\n"
+                      + "    \\\"f2\\\" frozen<udt_keyspace.field_with_timestamp>,\n"
+                      + "    \\\"f3\\\" frozen<udt_keyspace.field_with_timestamp>,\n"
+                      + "    \\\"f4\\\" frozen<udt_keyspace.field_with_timestamp>,\n"
+                      + "    \\\"f5\\\" frozen<udt_keyspace.field_with_timestamp>,\n"
+                      + "    \\\"f6\\\" frozen<udt_keyspace.field_with_timestamp>,\n"
+                      + "    \\\"f7\\\" frozen<udt_keyspace.field_with_timestamp>,\n"
+                      + "    \\\"f8\\\" frozen<udt_keyspace.field_with_timestamp>,\n"
+                      + "    \\\"f9\\\" text,\n"
+                      + "    \\\"f10\\\" frozen<map<bigint, frozen<map<text, boolean>>>>\n"
+                      + ");\n"
+                      + "CREATE TYPE another_keyspace.some_udt (\n"
+                      + "    x uuid,\n"
+                      + "    y text,\n"
+                      + "    z uuid\n"
+                      + ");\n"
+                      + "CREATE TYPE another_keyspace.another_udt (\n"
+                      + "    a uuid,\n"
+                      + "    b text,\n"
+                      + "    c uuid\n"
+                      + ");";
+        Set<String> udts = CqlUtils.extractUdts(schema, "udt_keyspace");
+        assertEquals(4, udts.size());
+        assertTrue(udts.contains("CREATE TYPE udt_keyspace.type_with_frozen_fields ("
+                               + "    \"f1\" frozen<udt_keyspace.field_with_timestamp>,"
+                               + "    \"f2\" frozen<udt_keyspace.field_with_timestamp>,"
+                               + "    \"f3\" frozen<udt_keyspace.field_with_timestamp>,"
+                               + "    \"f4\" frozen<udt_keyspace.field_with_timestamp>,"
+                               + "    \"f5\" frozen<udt_keyspace.field_with_timestamp>,"
+                               + "    \"f6\" frozen<udt_keyspace.field_with_timestamp>,"
+                               + "    \"f7\" frozen<udt_keyspace.field_with_timestamp>,"
+                               + "    \"f8\" frozen<udt_keyspace.field_with_timestamp>,"
+                               + "    \"f9\" text,"
+                               + "    \"f10\" frozen<map<bigint, frozen<map<text, boolean>>>>"
+                               + ");"));
+        assertTrue(udts.contains("CREATE TYPE udt_keyspace.type_with_time_zone ("
+                               + "    time bigint,"
+                               + "    \"timezoneOffsetMinutes\" int"
+                               + ");"));
+        assertTrue(udts.contains("CREATE TYPE udt_keyspace.type_1 ("
+                               + "    \"x\" text,"
+                               + "    \"y\" text,"
+                               + "    z text,"
+                               + "    \"a\" boolean"
+                               + ");"));
+        assertTrue(udts.contains("CREATE TYPE udt_keyspace.field_with_timestamp ("
+                               + "    field text,"
+                               + "    \"timeWithZone\" frozen<udt_keyspace.type_with_time_zone>"
+                               + ");"));
+    }
+
+    @Test
+    public void testExtractKeyspacesUDTs()
+    {
+        String schemaTxt = "\"CREATE KEYSPACE keyspace_with_udts WITH REPLICATION = {"
+                         + " 'class' : 'org.apache.cassandra.locator.NetworkTopologyStrategy',"
+                         + " 'datacenter1': '3' } AND DURABLE_WRITES = true;\\n\\n"
+                         + "CREATE TYPE keyspace_with_udts.type_with_time_zone (\\n"
+                         + "    time bigint,\\n"
+                         + "    \\\"timezoneOffsetMinutes\\\" int\\n);"
+                         + "CREATE TYPE keyspace_with_udts.field_with_timestamp (\\n"
+                         + "    field text,\\n"
+                         + "    \\\"timeWithZone\\\" frozen<keyspace_with_udts.type_with_time_zone>\\n);\\n\\n"
+                         + "CREATE TYPE keyspace_with_udts.type_with_frozen_fields (\\n"
+                         + "    \\\"f1\\\" frozen<keyspace_with_udts.field_with_timestamp>,\\n"
+                         + "    \\\"f2\\\" frozen<keyspace_with_udts.field_with_timestamp>,\\n"
+                         + "    \\\"f3\\\" frozen<map<bigint, int>>\\n);"
+                         + "CREATE KEYSPACE ks1 WITH REPLICATION = {"
+                         + " 'class' : 'org.apache.cassandra.locator.NetworkTopologyStrategy',"
+                         + " 'datacenter1': '3' } AND DURABLE_WRITES = true;\\n\\n"
+                         + "CREATE TYPE ks1.type_with_time_zone (\\n"
+                         + "    time bigint,\\n"
+                         + "    \\\"timezoneOffsetMinutes\\\" int\\n);\\n\\n"
+                         + "CREATE TYPE ks1.type_1 (\\n"
+                         + "    \\\"f1\\\" text,\\n"
+                         + "    \\\"f2\\\" text,\\n"
+                         + "    \\\"f3\\\" text\\n);\\n\\n"
+                         + "CREATE TYPE ks1.type_2 (\\n"
+                         + "    \\\"f1\\\" text,\\n"
+                         + "    \\\"f2\\\" text,\\n"
+                         + "    \\\"f3\\\" text,\\n"
+                         + "    f4 text\\n);\\n\\n"
+                         + "CREATE TYPE ks1.field_with_timestamp (\\n"
+                         + "    field text,\\n"
+                         + "    \\\"timeWithZone\\\" frozen<ks1.type_with_time_zone>\\n);";
+        Set<String> udts = CqlUtils.extractUdts(schemaTxt, "ks1");
+        assertEquals(4, udts.size());
+        String udtStr = String.join("\n", udts);
+        assertTrue(udtStr.contains("ks1.type_with_time_zone"));
+        assertTrue(udtStr.contains("ks1.type_1"));
+        assertTrue(udtStr.contains("ks1.type_2"));
+        assertTrue(udtStr.contains("ks1.field_with_timestamp"));
+    }
+
+    @Test
+    public void testExtractTableSchemaCase4()
+    {
+        String schemaStr = "CREATE TABLE keyspace.table (value text PRIMARY KEY);";
+        String expectedCreateStmt = "CREATE TABLE keyspace.table (value text PRIMARY KEY);";
+        String actualCreateStmt = CqlUtils.extractTableSchema(schemaStr, "keyspace", "table");
+        assert expectedCreateStmt.equals(actualCreateStmt);
+    }
+
+    @Test
+    public void testParseClusteringKeySchema()
+    {
+        String schemaTxt = "CREATE TABLE ks1.tb1 (\n"
+                         + "    namespace int,\n"
+                         + "    user_id text,\n"
+                         + "    dc_id int,\n"
+                         + "    ping_timestamp timestamp,\n"
+                         + "    PRIMARY KEY ((namespace, user_id), dc_id)\n"
+                         + ") WITH CLUSTERING ORDER BY (dc_id ASC)\n"
+                         + "    AND additional_write_policy = '99p'\n"
+                         + "    AND bloom_filter_fp_chance = 0.1\n"
+                         + "    AND caching = {'keys': 'ALL', 'rows_per_partition': 'NONE'}\n"
+                         + "    AND cdc = false\n"
+                         + "    AND comment = ''\n"
+                         + "    AND compaction = {'class': 'org.apache.cassandra.db.compaction.LeveledCompactionStrategy', "
+                         +                                "'max_threshold': '32', "
+                         +                                "'min_threshold': '4'}\n"
+                         + "    AND compression = {'chunk_length_in_kb': '16', "
+                         +                        "'class': 'org.apache.cassandra.io.compress.LZ4Compressor'}\n"
+                         + "    AND crc_check_chance = 1.0\n"
+                         + "    AND default_time_to_live = 0\n"
+                         + "    AND extensions = {}\n"
+                         + "    AND gc_grace_seconds = 864000\n"
+                         + "    AND max_index_interval = 256\n"
+                         + "    AND memtable_flush_period_in_ms = 0\n"
+                         + "    AND min_index_interval = 64\n"
+                         + "    AND read_repair = 'BLOCKING'\n"
+                         + "    AND speculative_retry = 'MIN(99p,15ms)';";
+        String actualCreateStmt = CqlUtils.extractTableSchema(schemaTxt, "ks1", "tb1");
+        bridge.buildSchema(actualCreateStmt, "ks1",
+                           new ReplicationFactor(ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy,
+                                                 ImmutableMap.of("datacenter1", 3)),
+                           Partitioner.Murmur3Partitioner,
+                           Collections.emptySet(), null, 0);
+    }
+
+    @Test
+    public void testExtractClusteringKey()
+    {
+        assertEquals("CLUSTERING ORDER BY (c ASC)",
+                     CqlUtils.extractClustering("CREATE TABLE ks1.tb1 (a int, b text, c int, d timestamp, PRIMARY KEY ((a, b), c)"
+                                              + " WITH CLUSTERING ORDER BY (c ASC) AND additional_write_policy = '99p';"));
+        assertEquals("CLUSTERING ORDER BY (c ASC)",
+                     CqlUtils.extractClustering("WITH CLUSTERING ORDER BY (c ASC)"));
+        assertEquals("CLUSTERING ORDER BY (c ASC)",
+                     CqlUtils.extractClustering("WITH CLUSTERING ORDER BY (c ASC);"));
+        assertEquals("CLUSTERING ORDER BY (c ASC)",
+                     CqlUtils.extractClustering("**** WITH CLUSTERING ORDER BY (c ASC)  AND ****     AND   ******* AND '***';"));
+        assertEquals("CLUSTERING ORDER BY (a DESC, b ASC, c ASC)",
+                     CqlUtils.extractClustering("CREATE TABLE ks1.tb1 (a int, b text, c int, d timestamp, PRIMARY KEY ((a, b), c)"
+                                              + " WITH CLUSTERING ORDER BY (a DESC, b ASC, c ASC) AND additional_write_policy = '99p'"
+                                              + " AND bloom_filter_fp_chance = 0.1 AND caching = {'keys': 'ALL', 'rows_per_partition': 'NONE'}"
+                                              + " AND cdc = false AND comment = '' AND compaction = {'class':"
+                                              + " 'org.apache.cassandra.db.compaction.LeveledCompactionStrategy', 'max_threshold': '32',"
+                                              + " 'min_threshold': '4'} AND compression = {'chunk_length_in_kb': '16', 'class':"
+                                              + " 'org.apache.cassandra.io.compress.LZ4Compressor'} AND crc_check_chance = 1.0"
+                                              + " AND default_time_to_live = 0 AND extensions = {} AND gc_grace_seconds = 864000"
+                                              + " AND max_index_interval = 256 AND memtable_flush_period_in_ms = 0 AND min_index_interval = 64"
+                                              + " AND read_repair = 'BLOCKING' AND speculative_retry = 'MIN(99p,15ms)';"));
+        assertEquals("CLUSTERING ORDER BY (a DESC, b ASC, c ASC)",
+                     CqlUtils.extractClustering("CREATE TABLE ks1.tb1 (a int, b text, c int, d timestamp, PRIMARY KEY ((a, b), c)"
+                                              + " WITH CLUSTERING ORDER BY (a DESC, b ASC, c ASC) AND additional_write_policy = '99p'"
+                                              + " AND bloom_filter_fp_chance = 0.1 AND caching = {'keys': 'ALL', 'rows_per_partition': 'NONE'}"
+                                              + " AND cdc = false AND comment = '' AND compaction = {'class':"
+                                              + " 'org.apache.cassandra.db.compaction.LeveledCompactionStrategy', 'max_threshold': '32',"
+                                              + " 'min_threshold': '4'} AND compression = {'chunk_length_in_kb': '16', 'class':"
+                                              + " 'org.apache.cassandra.io.compress.LZ4Compressor'} AND crc_check_chance = 1.0"
+                                              + " AND default_time_to_live = 0 AND extensions = {} AND gc_grace_seconds = 864000"
+                                              + " AND max_index_interval = 256 AND memtable_flush_period_in_ms = 0 AND min_index_interval = 64"
+                                              + " AND read_repair = 'BLOCKING' AND speculative_retry = 'MIN(99p,15ms)'"));
+        assertEquals("CLUSTERING ORDER BY (a DESC, b ASC, c ASC)",
+                     CqlUtils.extractClustering("WITH CLUSTERING ORDER BY (a DESC, b ASC, c ASC)"));
+        assertEquals("CLUSTERING ORDER BY (a ASC)",
+                     CqlUtils.extractClustering("CREATE TABLE ks1.tb1 (a int, b text, c int, d timestamp, PRIMARY KEY ((a, b), c))"
+                                              + " WITH CLUSTERING ORDER BY (a ASC) AND speculative_retry = 'MIN(99p,15ms);"));
+
+        assertNull(CqlUtils.extractClustering(""));
+        assertNull(CqlUtils.extractClustering("CREATE TABLE ks1.tb1 (a int, b text, c int, d timestamp, PRIMARY KEY ((a, b), c);"));
+        assertNull(CqlUtils.extractClustering("CREATE TABLE ks1.tb1 (a int, b text, c int, d timestamp, PRIMARY KEY ((a, b), c)"));
+        assertNull(CqlUtils.extractClustering("CREATE TABLE ks1.tb1 (a int, b text, c int, d timestamp, PRIMARY KEY ((a, b), c)"
+                                            + " AND additional_write_policy = '99p';"));
+        assertNull(CqlUtils.extractClustering("CREATE TABLE ks1.tb1 (a int, b text, c int, d timestamp, PRIMARY KEY ((a, b), c)"
+                                            + " AND additional_write_policy = '99p'"));
+        assertNull(CqlUtils.extractClustering("CREATE TABLE ks1.tb1 (a int, b text, c int, d timestamp, PRIMARY KEY ((a, b), c)"
+                                            + " AND additional_write_policy = '99p' AND bloom_filter_fp_chance = 0.1"
+                                            + " AND caching = {'keys': 'ALL', 'rows_per_partition': 'NONE'} AND cdc = false AND comment = ''"
+                                            + " AND compaction = {'class': 'org.apache.cassandra.db.compaction.LeveledCompactionStrategy',"
+                                            + " 'max_threshold': '32', 'min_threshold': '4'} AND compression = {'chunk_length_in_kb': '16',"
+                                            + " 'class': 'org.apache.cassandra.io.compress.LZ4Compressor'} AND crc_check_chance = 1.0"
+                                            + " AND default_time_to_live = 0 AND extensions = {} AND gc_grace_seconds = 864000 AND"
+                                            + " max_index_interval = 256 AND memtable_flush_period_in_ms = 0 AND min_index_interval = 64"
+                                            + " AND read_repair = 'BLOCKING' AND speculative_retry = 'MIN(99p,15ms)';"));
+        assertNull(CqlUtils.extractClustering("CREATE TABLE ks1.tb1 (a int, b text, c int, d timestamp, PRIMARY KEY ((a, b), c)"
+                                            + " AND additional_write_policy = '99p' AND bloom_filter_fp_chance = 0.1"
+                                            + " AND caching = {'keys': 'ALL', 'rows_per_partition': 'NONE'} AND cdc = false AND comment = ''"
+                                            + " AND compaction = {'class': 'org.apache.cassandra.db.compaction.LeveledCompactionStrategy',"
+                                            + " 'max_threshold': '32', 'min_threshold': '4'} AND compression = {'chunk_length_in_kb': '16',"
+                                            + " 'class': 'org.apache.cassandra.io.compress.LZ4Compressor'} AND crc_check_chance = 1.0"
+                                            + " AND default_time_to_live = 0 AND extensions = {} AND gc_grace_seconds = 864000 AND"
+                                            + " max_index_interval = 256 AND memtable_flush_period_in_ms = 0 AND min_index_interval = 64"
+                                            + " AND read_repair = 'BLOCKING' AND speculative_retry = 'MIN(99p,15ms)'"));
+    }
+
+    @Test
+    public void testClusteringOrderByIsRetained()
+    {
+        String schemaStr = "CREATE TABLE keyspace.table (id bigint, version bigint PRIMARY KEY (id, version)) "
+                         + "WITH CLUSTERING ORDER BY (id DESC, version DESC) AND foo = 1;";
+        String expectedCreateStmt = "CREATE TABLE keyspace.table (id bigint, version bigint PRIMARY KEY (id, version)) "
+                                  + "WITH CLUSTERING ORDER BY (id DESC, version DESC);";
+        String actualCreateStmt = CqlUtils.extractTableSchema(schemaStr, "keyspace", "table");
+        assertEquals(expectedCreateStmt, actualCreateStmt);
+    }
+
+    @Test
+    public void testExtractCdcFlag()
+    {
+        assertTrue(CqlUtils.extractOverrideProperties("CREATE TABLE k.t (a int PRIMARY KEY, b int);",
+                                                      Collections.singletonList("cdc")).isEmpty());
+        assertTrue(CqlUtils.extractOverrideProperties("CREATE TABLE k.t (a int PRIMARY KEY, b int) WITH cdc = true;",
+                                                      Collections.singletonList("cdc")).contains("cdc = true"));
+        assertTrue(CqlUtils.extractOverrideProperties("CREATE TABLE k.t (a int PRIMARY KEY, b int) WITH cdc = false;",
+                                                      Collections.singletonList("cdc")).contains("cdc = false"));
+        assertTrue(CqlUtils.extractOverrideProperties("CREATE TABLE k.t (a int PRIMARY KEY, b int) WITH cdc = true"
+                                                    + " AND caching = {'keys': 'ALL', 'rows_per_partition': 'NONE'};",
+                                                      Collections.singletonList("cdc")).contains("cdc = true"));
+        assertTrue(CqlUtils.extractOverrideProperties("CREATE TABLE k.t (a int PRIMARY KEY, b int) WITH cdc = false"
+                                                    + " AND caching = {'keys': 'ALL', 'rows_per_partition': 'NONE'};",
+                                                      Collections.singletonList("cdc")).contains("cdc = false"));
+        assertTrue(CqlUtils.extractOverrideProperties("CREATE TABLE k.t (a int PRIMARY KEY, b int) WITH bloom_filter_fp_chance = 0.1"
+                                                    + " AND caching = {'keys': 'ALL', 'rows_per_partition': 'NONE'} AND cdc = true;",
+                                                      Collections.singletonList("cdc")).contains("cdc = true"));
+        assertTrue(CqlUtils.extractOverrideProperties("CREATE TABLE k.t (a int PRIMARY KEY, b int) WITH bloom_filter_fp_chance = 0.1 "
+                                                    + "AND caching = {'keys': 'ALL', 'rows_per_partition': 'NONE'} AND cdc = false;",
+                                                      Collections.singletonList("cdc")).contains("cdc = false"));
+    }
+
+    @Test
+    public void testCdcExtractSchema()
+    {
+        String schema = "CREATE KEYSPACE ks1"
+                      + " WITH REPLICATION = { 'class' : 'org.apache.cassandra.locator.NetworkTopologyStrategy',"
+                      +                      " 'MS': '3', 'ST': '3' } AND DURABLE_WRITES = true;\\n\\n"
+                      + "CREATE TABLE ks1.tb1 (\"a\" text, \"b\" text, \"c\" text, \"d\" text,"
+                      + " e timestamp, f uuid, g blob, PRIMARY KEY ((\"a\", \"b\"), \"c\", \"d\"))"
+                      + " WITH CLUSTERING ORDER BY (\"c\" DESC, \"d\" ASC) AND cdc = true;\n\n"
+                      + "CREATE TABLE ks1.tb2 (a text, b text, c text, d text, PRIMARY KEY ((a), b, c))"
+                      + " WITH CLUSTERING ORDER BY (b DESC, c ASC) AND cdc = true;\n\n"
+                      + "CREATE KEYSPACE ks2"
+                      + " WITH REPLICATION = { 'class' : 'org.apache.cassandra.locator.NetworkTopologyStrategy',"
+                      +                      " 'MS': '3', 'ST': '3' } AND DURABLE_WRITES = true;\\n\\n"
+                      + "CREATE TABLE ks2.tb3 (a text, b text, c text, d text, PRIMARY KEY ((a), b, c))"
+                      + " WITH CLUSTERING ORDER BY (b DESC, c ASC);\n\n"
+                      + "CREATE TABLE ks2.tb4 (a bigint, b int, c uuid, d text, PRIMARY KEY (a)) WITH cdc = true;\n\n";
+        Map<TableIdentifier, String> createStmts = CqlUtils.extractCdcTables(schema);
+        assertEquals(3, createStmts.size());
+        assertTrue(createStmts.containsKey(TableIdentifier.of("ks1", "tb1")));
+        assertTrue(createStmts.containsKey(TableIdentifier.of("ks1", "tb2")));
+        assertFalse(createStmts.containsKey(TableIdentifier.of("ks2", "tb3")));
+        assertTrue(createStmts.containsKey(TableIdentifier.of("ks2", "tb4")));
+    }
+
+    private static String loadFullSchemaSample() throws URISyntaxException, IOException
+    {
+        File fullSchemaSampleFile =
+                new File(CqlUtilsTest.class.getClassLoader().getResource("cql/fullSchema.cql").toURI());
+        return FileUtils.readFileToString(fullSchemaSampleFile, StandardCharsets.UTF_8);
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/utils/FilterUtilsTests.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/utils/FilterUtilsTests.java
new file mode 100644
index 0000000..75e0276
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/utils/FilterUtilsTests.java
@@ -0,0 +1,80 @@
+/*
+ * 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.cassandra.spark.utils;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+
+import org.junit.Test;
+
+import org.apache.spark.sql.sources.EqualTo;
+import org.apache.spark.sql.sources.Filter;
+import org.apache.spark.sql.sources.In;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class FilterUtilsTests
+{
+    @Test(expected = IllegalArgumentException.class)
+    public void testPartialPartitionKeyFilter()
+    {
+        Filter[] filters = new Filter[]{new EqualTo("a", "1")};
+        Map<String, List<String>> partitionKeyValues = FilterUtils.extractPartitionKeyValues(filters, new HashSet<>(Arrays.asList("a", "b")));
+    }
+
+    @Test
+    public void testValidPartitionKeyValuesExtracted()
+    {
+        Filter[] filters = new Filter[]{new EqualTo("a", "1"), new In("b", new String[]{"2", "3"}), new EqualTo("c", "2")};
+        Map<String, List<String>> partitionKeyValues = FilterUtils.extractPartitionKeyValues(filters, new HashSet<>(Arrays.asList("a", "b")));
+        assertFalse(partitionKeyValues.containsKey("c"));
+        assertTrue(partitionKeyValues.containsKey("a"));
+        assertTrue(partitionKeyValues.containsKey("b"));
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void testCartesianProductOfInValidValues()
+    {
+        List<List<String>> orderedValues = Arrays.asList(Arrays.asList("1", "2"), Arrays.asList("a", "b", "c"), Collections.emptyList());
+        List<List<String>> product = FilterUtils.cartesianProduct(orderedValues);
+    }
+
+    @Test
+    public void testCartesianProductOfEmptyList()
+    {
+        List<List<String>> orderedValues = Collections.emptyList();
+        List<List<String>> product = FilterUtils.cartesianProduct(orderedValues);
+        assertFalse(product.isEmpty());
+        assertEquals(1, product.size());
+        assertTrue(product.get(0).isEmpty());
+    }
+
+    @Test
+    public void testCartesianProductOfSingleton()
+    {
+        List<List<String>> orderedValues = Collections.singletonList(Arrays.asList("a", "b", "c"));
+        assertEquals(3, FilterUtils.cartesianProduct(orderedValues).size());
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/utils/RangeUtilsTest.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/utils/RangeUtilsTest.java
new file mode 100644
index 0000000..9cba503
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/utils/RangeUtilsTest.java
@@ -0,0 +1,267 @@
+/*
+ * 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.cassandra.spark.utils;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import com.google.common.collect.Multimap;
+import com.google.common.collect.Range;
+import org.junit.Test;
+
+import org.apache.cassandra.spark.data.partitioner.CassandraInstance;
+import org.apache.cassandra.spark.data.partitioner.Partitioner;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class RangeUtilsTest
+{
+    private static final Pattern RANGE_PATTERN = Pattern.compile("^[\\\\(|\\[](-?\\d+),(-?\\d+)[\\\\)|\\]]$");
+
+    @Test
+    public void testCalculateTokenRangesTenNodesRF10()
+    {
+        assertTokenRanges(10, 10,
+                new String[]{"(-9223372036854775808,9223372036854775807]", "[-9223372036854775808,-9223372036854775808]"},
+                new String[]{"(-7378697629483820647,9223372036854775807]", "[-9223372036854775808,-7378697629483820647]"},
+                new String[]{"(-5534023222112865486,9223372036854775807]", "[-9223372036854775808,-5534023222112865486]"},
+                new String[]{"(-3689348814741910325,9223372036854775807]", "[-9223372036854775808,-3689348814741910325]"},
+                new String[]{"(-1844674407370955164,9223372036854775807]", "[-9223372036854775808,-1844674407370955164]"},
+                new String[]{"(-3,9223372036854775807]", "[-9223372036854775808,-3]"},
+                new String[]{"(1844674407370955158,9223372036854775807]", "[-9223372036854775808,1844674407370955158]"},
+                new String[]{"(3689348814741910319,9223372036854775807]", "[-9223372036854775808,3689348814741910319]"},
+                new String[]{"(5534023222112865480,9223372036854775807]", "[-9223372036854775808,5534023222112865480]"},
+                new String[]{"(7378697629483820641,9223372036854775807]", "[-9223372036854775808,7378697629483820641]"});
+    }
+
+    @Test
+    public void testCalculateTokenRangesTenNodesRF7()
+    {
+        assertTokenRanges(10, 7,
+                new String[]{"(-3689348814741910325,9223372036854775807]", "[-9223372036854775808,-9223372036854775808]"},
+                new String[]{"(-1844674407370955164,9223372036854775807]", "[-9223372036854775808,-7378697629483820647]"},
+                new String[]{"(-3,9223372036854775807]", "[-9223372036854775808,-5534023222112865486]"},
+                new String[]{"(1844674407370955158,9223372036854775807]", "[-9223372036854775808,-3689348814741910325]"},
+                new String[]{"(3689348814741910319,9223372036854775807]", "[-9223372036854775808,-1844674407370955164]"},
+                new String[]{"(5534023222112865480,9223372036854775807]", "[-9223372036854775808,-3]"},
+                new String[]{"(7378697629483820641,9223372036854775807]", "[-9223372036854775808,1844674407370955158]"},
+                new String[]{"(-9223372036854775808,3689348814741910319]"},
+                new String[]{"(-7378697629483820647,5534023222112865480]"},
+                new String[]{"(-5534023222112865486,7378697629483820641]"});
+    }
+
+    @Test
+    public void testCalculateTokenRangesTenNodesRF5()
+    {
+        assertTokenRanges(10, 5,
+                new String[]{"(-3,9223372036854775807]", "[-9223372036854775808,-9223372036854775808]"},
+                new String[]{"(1844674407370955158,9223372036854775807]", "[-9223372036854775808,-7378697629483820647]"},
+                new String[]{"(3689348814741910319,9223372036854775807]", "[-9223372036854775808,-5534023222112865486]"},
+                new String[]{"(5534023222112865480,9223372036854775807]", "[-9223372036854775808,-3689348814741910325]"},
+                new String[]{"(7378697629483820641,9223372036854775807]", "[-9223372036854775808,-1844674407370955164]"},
+                new String[]{"(-9223372036854775808,-3]"},
+                new String[]{"(-7378697629483820647,1844674407370955158]"},
+                new String[]{"(-5534023222112865486,3689348814741910319]"},
+                new String[]{"(-3689348814741910325,5534023222112865480]"},
+                new String[]{"(-1844674407370955164,7378697629483820641]"});
+    }
+
+    @Test
+    public void testCalculateTokenRangesTenNodesRF3()
+    {
+        assertTokenRanges(10, 3,
+                new String[]{"(3689348814741910319,9223372036854775807]", "[-9223372036854775808,-9223372036854775808]"},
+                new String[]{"(5534023222112865480,9223372036854775807]", "[-9223372036854775808,-7378697629483820647]"},
+                new String[]{"(7378697629483820641,9223372036854775807]", "[-9223372036854775808,-5534023222112865486]"},
+                new String[]{"(-9223372036854775808,-3689348814741910325]"},
+                new String[]{"(-7378697629483820647,-1844674407370955164]"},
+                new String[]{"(-5534023222112865486,-3]"},
+                new String[]{"(-3689348814741910325,1844674407370955158]"},
+                new String[]{"(-1844674407370955164,3689348814741910319]"},
+                new String[]{"(-3,5534023222112865480]"},
+                new String[]{"(1844674407370955158,7378697629483820641]"});
+    }
+
+    @Test
+    public void testCalculateTokenRangesTenNodesRF1()
+    {
+        assertTokenRanges(10, 1,
+                new String[]{"(7378697629483820641,9223372036854775807]", "[-9223372036854775808,-9223372036854775808]"},
+                new String[]{"(-9223372036854775808,-7378697629483820647]"},
+                new String[]{"(-7378697629483820647,-5534023222112865486]"},
+                new String[]{"(-5534023222112865486,-3689348814741910325]"},
+                new String[]{"(-3689348814741910325,-1844674407370955164]"},
+                new String[]{"(-1844674407370955164,-3]"},
+                new String[]{"(-3,1844674407370955158]"},
+                new String[]{"(1844674407370955158,3689348814741910319]"},
+                new String[]{"(3689348814741910319,5534023222112865480]"},
+                new String[]{"(5534023222112865480,7378697629483820641]"});
+    }
+
+    @Test
+    public void testCalculateTokenRangesFourNodesRF4()
+    {
+        assertTokenRanges(4, 4,
+                new String[]{"(-9223372036854775808,9223372036854775807]", "[-9223372036854775808,-9223372036854775808]"},
+                new String[]{"(-4611686018427387904,9223372036854775807]", "[-9223372036854775808,-4611686018427387904]"},
+                new String[]{"(0,9223372036854775807]", "[-9223372036854775808,0]"},
+                new String[]{"(4611686018427387904,9223372036854775807]", "[-9223372036854775808,4611686018427387904]"});
+    }
+
+    @Test
+    public void testCalculateTokenRangesFourNodesRF3()
+    {
+        assertTokenRanges(4, 3,
+                new String[]{"(-4611686018427387904,9223372036854775807]", "[-9223372036854775808,-9223372036854775808]"},
+                new String[]{"(0,9223372036854775807]", "[-9223372036854775808,-4611686018427387904]"},
+                new String[]{"(4611686018427387904,9223372036854775807]", "[-9223372036854775808,0]"},
+                new String[]{"(-9223372036854775808,4611686018427387904]"});
+    }
+
+    @Test
+    public void testCalculateTokenRangesFourNodesRF2()
+    {
+        assertTokenRanges(4, 2,
+                new String[]{"(0,9223372036854775807]", "[-9223372036854775808,-9223372036854775808]"},
+                new String[]{"(4611686018427387904,9223372036854775807]", "[-9223372036854775808,-4611686018427387904]"},
+                new String[]{"(-9223372036854775808,0]"},
+                new String[]{"(-4611686018427387904,4611686018427387904]"});
+    }
+
+    @Test
+    public void testCalculateTokenRangesFourNodesRF1()
+    {
+        assertTokenRanges(4, 1,
+                new String[]{"(4611686018427387904,9223372036854775807]", "[-9223372036854775808,-9223372036854775808]"},
+                new String[]{"(-9223372036854775808,-4611686018427387904]"},
+                new String[]{"(-4611686018427387904,0]"},
+                new String[]{"(0,4611686018427387904]"});
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void testCalculateTokenRangesRFGreaterThanNodesFails()
+    {
+        assertTokenRanges(2, 3,
+                new String[]{"Does Not"},
+                new String[]{"Matter"});
+        fail("Expected failure when RF greater than number of Nodes");
+    }
+
+    @Test
+    public void testCalculateTokenRangesZeroNodesSucceeds()
+    {
+        assertTokenRanges(0, 3);
+    }
+
+    private static void assertTokenRanges(int nodes, int replicationFactor, String[]... ranges)
+    {
+        assertEquals(nodes, ranges.length);
+        BigInteger[] tokens = getTokens(Partitioner.Murmur3Partitioner, nodes);
+        List<CassandraInstance> instances = getInstances(tokens);
+        Multimap<CassandraInstance, Range<BigInteger>> allRanges =
+                RangeUtils.calculateTokenRanges(instances, replicationFactor, Partitioner.Murmur3Partitioner);
+        for (int node = 0; node < nodes; node++)
+        {
+            assertExpectedRanges(allRanges.get(instances.get(node)), ranges[node]);
+        }
+    }
+
+    private static void assertExpectedRanges(Collection<Range<BigInteger>> actual, String... expectedRanges)
+    {
+        assertEquals(expectedRanges.length, actual.size());
+        for (String expected : expectedRanges)
+        {
+            assertTrue(String.format("Expected range %s not found in %s", expected, actual),
+                       actual.contains(range(expected)));
+        }
+    }
+
+    private static BigInteger[] getTokens(Partitioner partitioner, int nodes)
+    {
+        BigInteger[] tokens = new BigInteger[nodes];
+
+        for (int node = 0; node < nodes; node++)
+        {
+            tokens[node] = partitioner == Partitioner.Murmur3Partitioner
+                    ? getMurmur3Token(nodes, node)
+                    : getRandomToken(nodes, node);
+        }
+        return tokens;
+    }
+
+    private static BigInteger getRandomToken(int nodes, int index)
+    {
+        // ((2^127 / nodes) * i)
+        return ((BigInteger.valueOf(2).pow(127)).divide(BigInteger.valueOf(nodes))).multiply(BigInteger.valueOf(index));
+    }
+
+    private static BigInteger getMurmur3Token(int nodes, int index)
+    {
+        // (((2^64 / n) * i) - 2^63)
+        return (((BigInteger.valueOf(2).pow(64)).divide(BigInteger.valueOf(nodes)))
+                .multiply(BigInteger.valueOf(index))).subtract(BigInteger.valueOf(2).pow(63));
+    }
+
+    private static List<CassandraInstance> getInstances(BigInteger[] tokens)
+    {
+        List<CassandraInstance> instances = new ArrayList<>();
+        for (int token = 0; token < tokens.length; token++)
+        {
+            instances.add(new CassandraInstance(tokens[token].toString(), "node-" + token, "dc"));
+        }
+        return instances;
+    }
+
+    private static Range<BigInteger> range(String range)
+    {
+        Matcher m = RANGE_PATTERN.matcher(range);
+        if (m.matches())
+        {
+            int length = range.length();
+
+            BigInteger lowerBound = new BigInteger(m.group(1));
+            BigInteger upperBound = new BigInteger(m.group(2));
+
+            if (range.charAt(0) == '(')
+            {
+                if (range.charAt(length - 1) == ')')
+                {
+                    return Range.open(lowerBound, upperBound);
+                }
+                return Range.openClosed(lowerBound, upperBound);
+            }
+            else
+            {
+                if (range.charAt(length - 1) == ')')
+                {
+                    return Range.closedOpen(lowerBound, upperBound);
+                }
+                return Range.closed(lowerBound, upperBound);
+            }
+        }
+        throw new IllegalArgumentException("Range " + range + " is not valid");
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/utils/SSTableInputStreamHttpTests.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/utils/SSTableInputStreamHttpTests.java
new file mode 100644
index 0000000..1ab0584
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/utils/SSTableInputStreamHttpTests.java
@@ -0,0 +1,312 @@
+/*
+ * 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.cassandra.spark.utils;
+
+import java.io.BufferedOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.RandomAccessFile;
+import java.net.InetSocketAddress;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.security.MessageDigest;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.commons.codec.digest.DigestUtils;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.sun.net.httpserver.HttpServer;
+import org.apache.cassandra.spark.data.FileType;
+import org.apache.cassandra.spark.data.SSTable;
+import org.apache.cassandra.spark.utils.streaming.SSTableInputStream;
+import org.apache.cassandra.spark.utils.streaming.SSTableSource;
+import org.apache.cassandra.spark.utils.streaming.StreamBuffer;
+import org.apache.cassandra.spark.utils.streaming.StreamConsumer;
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.impl.client.HttpClients;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.quicktheories.QuickTheory.qt;
+import static org.quicktheories.generators.SourceDSL.arbitrary;
+
+/**
+ * Test the {@link SSTableInputStream} by firing up an in-test HTTP server and reading the files with an HTTP client
+ * Compares the MD5s to verify file bytes match bytes returned by {@link SSTableInputStream}.
+ */
+public class SSTableInputStreamHttpTests
+{
+    static final ExecutorService HTTP_EXECUTOR =
+            Executors.newFixedThreadPool(4, new ThreadFactoryBuilder().setNameFormat("http-server-%d")
+                                                                      .setDaemon(true)
+                                                                      .build());
+    static final ExecutorService HTTP_CLIENT =
+            Executors.newFixedThreadPool(4, new ThreadFactoryBuilder().setNameFormat("http-client-%d")
+                                                                      .setDaemon(true)
+                                                                      .build());
+    private static final Logger LOGGER = LoggerFactory.getLogger(SSTableInputStreamHttpTests.class);
+
+    @ClassRule
+    public static TemporaryFolder DIRECTORY = new TemporaryFolder();  // CHECKSTYLE IGNORE: Constant cannot be made final
+    private static final String HOST = "localhost";
+    private static final int PORT = 8001;
+    private static final int HTTP_CLIENT_CHUNK_SIZE = 8192;
+
+    private static HttpServer SERVER;           // CHECKSTYLE IGNORE: Constant cannot be made final
+    private static CloseableHttpClient CLIENT;  // CHECKSTYLE IGNORE: Constant cannot be made final
+
+    @BeforeClass
+    public static void setup() throws IOException
+    {
+        // Create in-test HTTP server to handle range requests and tmp files
+        SERVER = HttpServer.create(new InetSocketAddress(HOST, PORT), 0);
+        SERVER.setExecutor(HTTP_EXECUTOR);
+        SERVER.createContext("/", exchange -> {
+            try
+            {
+                String uri = exchange.getRequestURI().getPath();
+                Path path = Paths.get(DIRECTORY.getRoot().getPath(), uri);
+
+                // Extract Range from header
+                long size = Files.size(path);
+                long start;
+                long end;
+                String rangeHeader = exchange.getRequestHeaders().getFirst("Range");
+                if (rangeHeader != null)
+                {
+                    String[] range = rangeHeader.split("=")[1].split("-");
+                    start = Long.parseLong(range[0]);
+                    long endValue = Long.parseLong(range[1]);
+                    if (endValue < start)
+                    {
+                        exchange.sendResponseHeaders(416, -1);
+                        return;
+                    }
+                    end = Math.min(size - 1, endValue);
+                }
+                else
+                {
+                    start = 0;
+                    end = size;
+                }
+
+                // Return file bytes within range
+                int length = (int) (end - start + 1);
+                if (length <= 0)
+                {
+                    exchange.sendResponseHeaders(200, -1);
+                    return;
+                }
+
+                LOGGER.info("Serving file filename={} start={} end={} length={}",
+                            path.getFileName(), start, end, length);
+                exchange.sendResponseHeaders(200, length);
+                try (RandomAccessFile raf = new RandomAccessFile(path.toFile(), "r"))
+                {
+                    byte[] bytes = new byte[length];
+                    raf.seek(start);
+                    raf.read(bytes, 0, length);
+                    exchange.getResponseBody().write(bytes);
+                }
+                exchange.getResponseBody().flush();
+            }
+            catch (Throwable throwable)
+            {
+                LOGGER.error("Unexpected exception in in-test HTTP server", throwable);
+                exchange.sendResponseHeaders(500, -1);
+            }
+            finally
+            {
+                exchange.close();
+            }
+        });
+        SERVER.start();
+        LOGGER.info("Started in-test HTTP Server host={} port={}", HOST, PORT);
+
+        CLIENT = HttpClients.createDefault();
+    }
+
+    @AfterClass
+    public static void tearDown()
+    {
+        SERVER.stop(0);
+    }
+
+    // HTTP client source for reading SSTable from HTTP server
+    private static SSTableSource<SSTable> buildHttpSource(String filename,
+                                                          long size,
+                                                          Long maxBufferSize,
+                                                          Long chunkBufferSize)
+    {
+        return new SSTableSource<SSTable>()
+        {
+            public void request(long start, long end, StreamConsumer consumer)
+            {
+                CompletableFuture.runAsync(() -> {
+                    LOGGER.info("Reading file using HTTP client filename={} start={} end={}", filename, start, end);
+                    HttpGet get = new HttpGet("http://" + HOST + ":" + PORT + "/" + filename);
+                    get.setHeader("Range", String.format("bytes=%d-%d", start, end));
+                    try
+                    {
+                        CloseableHttpResponse resp = CLIENT.execute(get);
+                        if (resp.getStatusLine().getStatusCode() == 200)
+                        {
+                            try (InputStream is = resp.getEntity().getContent())
+                            {
+                                int length;
+                                byte[] bytes = new byte[HTTP_CLIENT_CHUNK_SIZE];
+                                while ((length = is.read(bytes, 0, bytes.length)) >= 0)
+                                {
+                                    if (length < HTTP_CLIENT_CHUNK_SIZE)
+                                    {
+                                        byte[] copy = new byte[length];
+                                        System.arraycopy(bytes, 0, copy, 0, length);
+                                        bytes = copy;
+                                    }
+                                    consumer.onRead(StreamBuffer.wrap(bytes));
+                                    bytes = new byte[HTTP_CLIENT_CHUNK_SIZE];
+                                }
+                                consumer.onEnd();
+                            }
+                        }
+                        else
+                        {
+                            consumer.onError(new RuntimeException("Unexpected status code: " + resp.getStatusLine().getStatusCode()));
+                        }
+                    }
+                    catch (IOException exception)
+                    {
+                        consumer.onError(exception);
+                    }
+                }, HTTP_CLIENT);
+            }
+
+            public long maxBufferSize()
+            {
+                return maxBufferSize != null ? maxBufferSize : SSTableSource.super.maxBufferSize();
+            }
+
+            public long chunkBufferSize()
+            {
+                return chunkBufferSize != null ? chunkBufferSize : SSTableSource.super.chunkBufferSize();
+            }
+
+            public SSTable sstable()
+            {
+                return null;
+            }
+
+            public FileType fileType()
+            {
+                return null;
+            }
+
+            public long size()
+            {
+                return size;
+            }
+        };
+    }
+
+    // Tests
+
+    @Test
+    public void testSmallChunkSizes()
+    {
+        List<Long> fileSizes = Arrays.asList(1L, 1536L, 32768L, 250000L, 10485800L);
+        qt().forAll(arbitrary().pick(fileSizes)).checkAssert(size -> runHttpTest(size, 8192L, 4096L));
+    }
+
+    @Test
+    public void testDefaultClientConfig()
+    {
+        List<Long> fileSizes = Arrays.asList(1L, 13L, 512L, 1024L, 1536L, 32768L, 1000000L, 10485800L, 1000000000L);
+        qt().forAll(arbitrary().pick(fileSizes)).checkAssert(this::runHttpTest);
+    }
+
+    // HTTP Tester
+
+    private void runHttpTest(long size)
+    {
+        runHttpTest(size, null, null);
+    }
+
+    // Test SSTableInputStream against test HTTP server using HTTP client
+    private void runHttpTest(long size, Long maxBufferSize, Long chunkBufferSize)
+    {
+        try
+        {
+            // Create tmp file with random data
+            Path path = DIRECTORY.newFile().toPath();
+            MessageDigest digest = DigestUtils.getMd5Digest();
+            try (BufferedOutputStream out = new BufferedOutputStream(Files.newOutputStream(path)))
+            {
+                long remaining = size;
+                while (remaining > 0)
+                {
+                    byte[] bytes = RandomUtils.randomBytes((int) Math.min(remaining, SSTableInputStreamTests.DEFAULT_CHUNK_SIZE));
+                    out.write(bytes);
+                    digest.update(bytes);
+                    remaining -= bytes.length;
+                }
+            }
+            byte[] expectedMD5 = digest.digest();
+            assertEquals(size, Files.size(path));
+            LOGGER.info("Created random file path={} fileSize={}", path, size);
+
+            // Use HTTP client source to read files across HTTP and verify MD5 matches expected
+            byte[] actualMD5;
+            long blockingTimeMillis;
+            SSTableSource<SSTable> source = buildHttpSource(path.getFileName().toString(),
+                                                            Files.size(path),
+                                                            maxBufferSize,
+                                                            chunkBufferSize);
+            try (SSTableInputStream<SSTable> is = new SSTableInputStream<>(source, SSTableInputStreamTests.STATS))
+            {
+                actualMD5 = DigestUtils.md5(is);
+                blockingTimeMillis = TimeUnit.MILLISECONDS.convert(is.timeBlockedNanos(), TimeUnit.NANOSECONDS);
+                assertEquals(size, is.bytesRead());
+                assertEquals(0L, is.bytesBuffered());
+            }
+            LOGGER.info("Time spent blocking on InputStream thread blockingTimeMillis={} fileSize={}",
+                        blockingTimeMillis, size);
+            assertArrayEquals(actualMD5, expectedMD5);
+        }
+        catch (IOException exception)
+        {
+            throw new RuntimeException(exception);
+        }
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/utils/SSTableInputStreamTests.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/utils/SSTableInputStreamTests.java
new file mode 100644
index 0000000..bb2b656
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/utils/SSTableInputStreamTests.java
@@ -0,0 +1,396 @@
+/*
+ * 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.cassandra.spark.utils;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import com.google.common.util.concurrent.Uninterruptibles;
+import org.apache.commons.lang.mutable.MutableInt;
+import org.junit.Test;
+
+import org.apache.cassandra.spark.data.FileType;
+import org.apache.cassandra.spark.data.SSTable;
+import org.apache.cassandra.spark.stats.Stats;
+import org.apache.cassandra.spark.utils.streaming.SSTableInputStream;
+import org.apache.cassandra.spark.utils.streaming.SSTableSource;
+import org.apache.cassandra.spark.utils.streaming.StreamBuffer;
+import org.apache.cassandra.spark.utils.streaming.StreamConsumer;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.cassandra.spark.utils.streaming.SSTableInputStream.timeoutLeftNanos;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * Test the {@link SSTableInputStream} by mocking the {@link SSTableSource}
+ */
+public class SSTableInputStreamTests
+{
+    private static final ScheduledExecutorService SCHEDULER = Executors.newScheduledThreadPool(1);
+    private static final ExecutorService EXECUTOR =
+            Executors.newFixedThreadPool(4, new ThreadFactoryBuilder().setNameFormat("sstable-tests-%d")
+                                                                      .setDaemon(true)
+                                                                      .build());
+    static final int DEFAULT_CHUNK_SIZE = 8192;
+    static final Stats STATS = Stats.DoNothingStats.INSTANCE;
+
+    // Mocked Tests
+
+    @Test
+    public void testMockedClient() throws IOException
+    {
+        runMockedTest(1, 1, DEFAULT_CHUNK_SIZE);
+        runMockedTest(1, 5, DEFAULT_CHUNK_SIZE * 5);
+        runMockedTest(10, 10, SSTableSource.DEFAULT_MAX_BUFFER_SIZE);
+        runMockedTest(20, 1024, 33554400L);
+        runMockedTest(10, 10, DEFAULT_CHUNK_SIZE * 10);
+    }
+
+    private interface SSTableRequest
+    {
+        void request(long start, long end, StreamConsumer consumer);
+    }
+
+    private static SSTableSource<SSTable> buildSource(long size,
+                                                      Long maxBufferSize,
+                                                      Long requestChunkSize,
+                                                      SSTableRequest request,
+                                                      Duration duration)
+    {
+        return new SSTableSource<SSTable>()
+        {
+            public void request(long start, long end, StreamConsumer consumer)
+            {
+                request.request(start, end, consumer);
+            }
+
+            public SSTable sstable()
+            {
+                return null;
+            }
+
+            public FileType fileType()
+            {
+                return null;
+            }
+
+            public long size()
+            {
+                return size;
+            }
+
+            public long maxBufferSize()
+            {
+                return maxBufferSize != null ? maxBufferSize : SSTableSource.DEFAULT_MAX_BUFFER_SIZE;
+            }
+
+            public long chunkBufferSize()
+            {
+                return requestChunkSize != null ? requestChunkSize : SSTableSource.DEFAULT_CHUNK_BUFFER_SIZE;
+            }
+
+            public Duration timeout()
+            {
+                return duration;
+            }
+        };
+    }
+
+    // Test SSTableInputStream using mocked SSTableSource
+    private void runMockedTest(int numRequests, int chunksPerRequest, long maxBufferSize) throws IOException
+    {
+        long requestChunkSize = (long) DEFAULT_CHUNK_SIZE * chunksPerRequest;
+        long fileSize = requestChunkSize * (long) numRequests;
+        AtomicInteger requestCount = new AtomicInteger(0);
+        SSTableSource<SSTable> mockedClient = buildSource(fileSize,
+                                                          maxBufferSize,
+                                                          requestChunkSize,
+                                                          (start, end, consumer) -> {
+            requestCount.incrementAndGet();
+            writeBuffers(consumer, randomBuffers(chunksPerRequest));
+        }, null);
+        SSTableInputStream<SSTable> is = new SSTableInputStream<>(mockedClient, STATS);
+        readStreamFully(is);
+        assertEquals(numRequests, requestCount.get());
+        assertEquals(0L, is.bytesBuffered());
+        assertEquals(fileSize, is.bytesWritten());
+        assertEquals(fileSize, is.bytesRead());
+    }
+
+    @Test(expected = IOException.class)
+    public void testFailure() throws IOException
+    {
+        int chunksPerRequest = 10;
+        int numRequests = 10;
+        long length = SSTableSource.DEFAULT_CHUNK_BUFFER_SIZE * chunksPerRequest * numRequests;
+        AtomicInteger count = new AtomicInteger(0);
+        SSTableSource<SSTable> source = buildSource(length,
+                                                    SSTableSource.DEFAULT_MAX_BUFFER_SIZE,
+                                                    SSTableSource.DEFAULT_CHUNK_BUFFER_SIZE,
+                                                    (start, end, consumer) -> {
+            if (count.incrementAndGet() > (numRequests / 2))
+            {
+                // Halfway through throw random exception
+                EXECUTOR.submit(() -> consumer.onError(new RuntimeException("Something bad happened...")));
+            }
+            else
+            {
+                writeBuffers(consumer, randomBuffers(chunksPerRequest));
+            }
+        }, null);
+        readStreamFully(new SSTableInputStream<>(source, STATS));
+        fail("Should have failed with IOException");
+    }
+
+    @Test
+    public void testTimeout()
+    {
+        long now = System.nanoTime();
+        assertEquals(Duration.ofMillis(100).toNanos(),
+                     timeoutLeftNanos(Duration.ofMillis(1000), now, now - Duration.ofMillis(900).toNanos()));
+        assertEquals(Duration.ofMillis(-500).toNanos(),
+                     timeoutLeftNanos(Duration.ofMillis(1000), now, now - Duration.ofMillis(1500).toNanos()));
+        assertEquals(Duration.ofMillis(995).toNanos(),
+                     timeoutLeftNanos(Duration.ofMillis(1000), now, now - Duration.ofMillis(5).toNanos()));
+        assertEquals(Duration.ofMillis(1000).toNanos(),
+                     timeoutLeftNanos(Duration.ofMillis(1000), now, now - Duration.ofMillis(0).toNanos()));
+        assertEquals(Duration.ofMillis(1000).toNanos(),
+                     timeoutLeftNanos(Duration.ofMillis(1000), now, now + Duration.ofMillis(500).toNanos()));
+        assertEquals(Duration.ofMillis(35000).toNanos(),
+                     timeoutLeftNanos(Duration.ofMillis(60000), now, now - Duration.ofMillis(25000).toNanos()));
+        assertEquals(Duration.ofMillis(-5000).toNanos(),
+                     timeoutLeftNanos(Duration.ofMillis(60000), now, now - Duration.ofMillis(65000).toNanos()));
+        assertEquals(Duration.ofMillis(0).toNanos(),
+                     timeoutLeftNanos(Duration.ofMillis(60000), now, now - Duration.ofMillis(60000).toNanos()));
+    }
+
+    @Test
+    @SuppressWarnings("UnstableApiUsage")
+    public void testTimeoutShouldAccountForActivityTime()
+    {
+        int chunksPerRequest = 10;
+        int numRequests = 10;
+        long length = SSTableSource.DEFAULT_CHUNK_BUFFER_SIZE * chunksPerRequest * numRequests;
+        AtomicInteger count = new AtomicInteger(0);
+        Duration timeout = Duration.ofMillis(1000);
+        long startTime = System.nanoTime();
+        long sleepTimeInMillis = 100L;
+        SSTableSource<SSTable> source = buildSource(length,
+                                                    SSTableSource.DEFAULT_MAX_BUFFER_SIZE,
+                                                    SSTableSource.DEFAULT_CHUNK_BUFFER_SIZE,
+                                                    (start, end, consumer) -> {
+            // Only respond once so future requests will time out
+            if (count.incrementAndGet() == 1)
+            {
+                EXECUTOR.submit(() -> {
+                    Uninterruptibles.sleepUninterruptibly(sleepTimeInMillis, TimeUnit.MILLISECONDS);
+                    writeBuffers(consumer, randomBuffers(chunksPerRequest));
+                });
+            }
+        }, timeout);
+        try
+        {
+            readStreamFully(new SSTableInputStream<>(source, STATS));
+            fail("Should not reach here, should throw TimeoutException");
+        }
+        catch (IOException exception)
+        {
+            assertTrue(exception.getCause() instanceof TimeoutException);
+        }
+        Duration duration = Duration.ofNanos(System.nanoTime() - startTime);
+        Duration maxAcceptable = timeout.plus(Duration.ofMillis(sleepTimeInMillis));
+        assertTrue("Timeout didn't account for activity time. "
+                 + "Took " + duration.toMillis() + "ms should have taken at most " + maxAcceptable.toMillis() + "ms",
+                   duration.minus(maxAcceptable).toMillis() < 100);
+    }
+
+    @Test
+    public void testSkipOnInit() throws IOException
+    {
+        int size = 20971520;
+        int chunkSize = 1024;
+        int numChunks = 16;
+        MutableInt bytesRead = new MutableInt(0);
+        MutableInt count = new MutableInt(0);
+        SSTableSource<SSTable> source = new SSTableSource<SSTable>()
+        {
+            @Override
+            public void request(long start, long end, StreamConsumer consumer)
+            {
+                assertNotEquals(0, start);
+                int length = (int) (end - start + 1);
+                consumer.onRead(randomBuffer(length));
+                bytesRead.add(length);
+                count.increment();
+                consumer.onEnd();
+            }
+
+            @Override
+            public long chunkBufferSize()
+            {
+                return chunkSize;
+            }
+
+            @Override
+            public SSTable sstable()
+            {
+                return null;
+            }
+
+            @Override
+            public FileType fileType()
+            {
+                return FileType.INDEX;
+            }
+
+            @Override
+            public long size()
+            {
+                return size;
+            }
+
+            @Override
+            @Nullable
+            public Duration timeout()
+            {
+                return Duration.ofSeconds(5);
+            }
+        };
+
+        int bytesToRead = chunkSize * numChunks;
+        long skipAhead = size - bytesToRead + 1;
+        try (SSTableInputStream<SSTable> stream = new SSTableInputStream<>(source, STATS))
+        {
+            // Skip ahead so we only read the final chunks
+            ByteBufferUtils.skipFully(stream, skipAhead);
+            readStreamFully(stream);
+        }
+        // Verify we only read final chunks and not the start of the file
+        assertEquals(bytesToRead, bytesRead.intValue());
+        assertEquals(numChunks, count.intValue());
+    }
+
+    @Test
+    public void testSkipToEnd() throws IOException
+    {
+        SSTableSource<SSTable> source = new SSTableSource<SSTable>()
+        {
+            @Override
+            public void request(long start, long end, StreamConsumer consumer)
+            {
+                consumer.onRead(randomBuffer((int) (end - start + 1)));
+                consumer.onEnd();
+            }
+
+            @Override
+            public SSTable sstable()
+            {
+                return null;
+            }
+
+            @Override
+            public FileType fileType()
+            {
+                return FileType.INDEX;
+            }
+
+            @Override
+            public long size()
+            {
+                return 20971520;
+            }
+
+            @Override
+            @Nullable
+            public Duration timeout()
+            {
+                return Duration.ofSeconds(5);
+            }
+        };
+
+        try (SSTableInputStream<SSTable> stream = new SSTableInputStream<>(source, STATS))
+        {
+            ByteBufferUtils.skipFully(stream, 20971520);
+            readStreamFully(stream);
+        }
+    }
+
+    // Utils
+
+    private static ImmutableList<StreamBuffer> randomBuffers(int count)
+    {
+        return ImmutableList.copyOf(IntStream.range(0, count)
+                                             .mapToObj(buffer -> randomBuffer())
+                                             .collect(Collectors.toList()));
+    }
+
+    private static StreamBuffer randomBuffer()
+    {
+        return randomBuffer(DEFAULT_CHUNK_SIZE);
+    }
+
+    private static StreamBuffer randomBuffer(int size)
+    {
+        return StreamBuffer.wrap(RandomUtils.randomBytes(size));
+    }
+
+    @SuppressWarnings("StatementWithEmptyBody")
+    private static void readStreamFully(SSTableInputStream<SSTable> inputStream) throws IOException
+    {
+        try (SSTableInputStream<SSTable> in = inputStream)
+        {
+            while (in.read() >= 0)
+            {
+                // CHECKSTYLE IGNORE: Do nothing
+            }
+        }
+    }
+
+    private static void writeBuffers(StreamConsumer consumer, ImmutableList<StreamBuffer> buffers)
+    {
+        if (buffers.isEmpty())
+        {
+            // No more buffers so finished
+            consumer.onEnd();
+            return;
+        }
+
+        SCHEDULER.schedule(() -> {
+            EXECUTOR.submit(() -> {
+                // Write next buffer to StreamConsumer
+                consumer.onRead(buffers.get(0));
+                writeBuffers(consumer, buffers.subList(1, buffers.size()));
+            });
+        }, RandomUtils.RANDOM.nextInt(50), TimeUnit.MICROSECONDS);  // Inject random latency
+    }
+}
diff --git a/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/utils/SerializationUtils.java b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/utils/SerializationUtils.java
new file mode 100644
index 0000000..47273e7
--- /dev/null
+++ b/cassandra-analytics-core/src/test/java/org/apache/cassandra/spark/utils/SerializationUtils.java
@@ -0,0 +1,97 @@
+/*
+ * 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.cassandra.spark.utils;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.Serializer;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+
+/**
+ * A test helper class for serialization/deserialization methods
+ */
+public final class SerializationUtils
+{
+    private static final Kryo KRYO = new Kryo();
+
+    private SerializationUtils()
+    {
+        throw new IllegalStateException(getClass() + " is static utility class and shall not be instantiated");
+    }
+
+    public static void register(Class<?> type, Serializer<?> serializer)
+    {
+        KRYO.register(type, serializer);
+    }
+
+    public static byte[] serialize(Object object)
+    {
+        try
+        {
+            ByteArrayOutputStream arOut = new ByteArrayOutputStream(512);
+            try (ObjectOutputStream out = new ObjectOutputStream(arOut))
+            {
+                out.writeObject(object);
+            }
+            return arOut.toByteArray();
+        }
+        catch (IOException exception)
+        {
+            throw new RuntimeException(exception);
+        }
+    }
+
+    public static <T> T deserialize(byte[] bytes, Class<T> type)
+    {
+        ObjectInputStream in;
+        try
+        {
+            in = new ObjectInputStream(new ByteArrayInputStream(bytes));
+            return type.cast(in.readObject());
+        }
+        catch (IOException | ClassNotFoundException exception)
+        {
+            throw new RuntimeException(exception);
+        }
+    }
+
+    public static Output kryoSerialize(Object object)
+    {
+        try (Output out = new Output(1024, -1))
+        {
+            KRYO.writeObject(out, object);
+            return out;
+        }
+    }
+
+    public static <T> T kryoDeserialize(Output out, Class<T> type)
+    {
+        try (Input in = new Input(out.getBuffer(), 0, out.position()))
+        {
+            return KRYO.readObject(in, type);
+        }
+    }
+}
diff --git a/cassandra-analytics-core/src/test/resources/cql/fullSchema.cql b/cassandra-analytics-core/src/test/resources/cql/fullSchema.cql
new file mode 100644
index 0000000..38be7e6
--- /dev/null
+++ b/cassandra-analytics-core/src/test/resources/cql/fullSchema.cql
@@ -0,0 +1,105 @@
+CREATE KEYSPACE "quoted_keyspace" WITH REPLICATION = { 'class' : 'org.apache.cassandra.locator.NetworkTopologyStrategy', 'datacenter1': '3', 'datacenter2': '3' } AND DURABLE_WRITES = true;
+
+CREATE KEYSPACE keyspace WITH REPLICATION = { 'class' : 'org.apache.cassandra.locator.NetworkTopologyStrategy', 'datacenter1': '4', 'datacenter2': '3' } AND DURABLE_WRITES = true;
+
+CREATE TABLE keyspace.tbl1 (
+                               id uuid,
+                               foo uuid,
+                               created_at timestamp,
+                               group text,
+                               PRIMARY KEY (id)
+) WITH gc_grace_seconds = 864000
+      AND bloom_filter_fp_chance = 0.1
+      AND caching = { 'keys' : 'ALL', 'rows_per_partition' : 'NONE' }
+      AND comment = ''
+      AND compaction = { 'class' : 'org.apache.cassandra.db.compaction.LeveledCompactionStrategy', 'max_threshold' : 32, 'min_threshold' : 4 }
+      AND compression = { 'chunk_length_in_kb' : 64, 'class' : 'org.apache.cassandra.io.compress.LZ4Compressor' }
+      AND default_time_to_live = 0
+      AND speculative_retry = '99p'
+      AND min_index_interval = 128
+      AND max_index_interval = 2048
+      AND crc_check_chance = 1.0
+      AND cdc = false
+      AND memtable_flush_period_in_ms = 0;
+
+CREATE INDEX tbl1_foo ON keyspace.tbl1 (foo);
+
+CREATE INDEX tbl1_group_idx ON keyspace.tbl1 (group);
+
+CREATE TABLE keyspace.tbl2 (
+                               id uuid,
+                               created_at timestamp,
+                               hosts text,
+                               PRIMARY KEY (id)
+) WITH gc_grace_seconds = 864000
+      AND bloom_filter_fp_chance = 0.1
+      AND caching = { 'keys' : 'ALL', 'rows_per_partition' : 'NONE' }
+      AND comment = ''
+      AND compaction = { 'class' : 'org.apache.cassandra.db.compaction.LeveledCompactionStrategy', 'max_threshold' : 32, 'min_threshold' : 4 }
+      AND compression = { 'chunk_length_in_kb' : 64, 'class' : 'org.apache.cassandra.io.compress.LZ4Compressor' }
+      AND default_time_to_live = 0
+      AND speculative_retry = '99p'
+      AND min_index_interval = 128
+      AND max_index_interval = 2048
+      AND crc_check_chance = 1.0
+      AND cdc = false
+      AND memtable_flush_period_in_ms = 0;
+
+CREATE KEYSPACE system_distributed WITH REPLICATION = { 'class' : 'org.apache.cassandra.locator.SimpleStrategy', 'replication_factor': '3' } AND DURABLE_WRITES = true;
+
+CREATE KEYSPACE system_schema WITH REPLICATION = { 'class' : 'org.apache.cassandra.locator.LocalStrategy' } AND DURABLE_WRITES = true;
+
+CREATE KEYSPACE cycling WITH REPLICATION = { 'class' : 'org.apache.cassandra.locator.NetworkTopologyStrategy', 'datacenter1': '3' } AND DURABLE_WRITES = true;
+
+CREATE TABLE cycling.rank_by_year_and_name (
+                                               race_year int,
+                                               race_name text,
+                                               rank int,
+                                               cyclist_name text,
+                                               PRIMARY KEY ((race_year, race_name), rank)
+) WITH CLUSTERING ORDER BY (rank ASC)
+   AND read_repair = 'BLOCKING'
+   AND gc_grace_seconds = 864000
+   AND additional_write_policy = '99p'
+   AND bloom_filter_fp_chance = 0.01
+   AND caching = { 'keys' : 'ALL', 'rows_per_partition' : 'NONE' }
+   AND comment = ''
+   AND compaction = { 'class' : 'org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy', 'max_threshold' : 32, 'min_threshold' : 4 }
+   AND compression = { 'chunk_length_in_kb' : 16, 'class' : 'org.apache.cassandra.io.compress.LZ4Compressor' }
+   AND default_time_to_live = 0
+   AND speculative_retry = '99p'
+   AND min_index_interval = 128
+   AND max_index_interval = 2048
+   AND crc_check_chance = 1.0
+   AND cdc = false
+   AND memtable_flush_period_in_ms = 0;
+
+CREATE TABLE cycling.rank_by_year_and_name_quoted_columns
+(
+    race_year      int,
+    "RACE_NAME"    text,
+    rank           int,
+    "cyclist_Name" text,
+    PRIMARY KEY ((race_year, "RACE_NAME"), rank)
+) WITH CLUSTERING ORDER BY (rank ASC)
+   AND read_repair = 'BLOCKING'
+   AND gc_grace_seconds = 864000
+   AND additional_write_policy = '99p'
+   AND bloom_filter_fp_chance = 0.01
+   AND caching = { 'keys' : 'ALL', 'rows_per_partition' : 'NONE' }
+   AND comment = ''
+   AND compaction = { 'class' : 'org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy', 'max_threshold' : 32, 'min_threshold' : 4 }
+   AND compression = { 'chunk_length_in_kb' : 16, 'class' : 'org.apache.cassandra.io.compress.LZ4Compressor' }
+   AND default_time_to_live = 0
+   AND speculative_retry = '99p'
+   AND min_index_interval = 128
+   AND max_index_interval = 2048
+   AND crc_check_chance = 1.0
+   AND cdc = false
+   AND memtable_flush_period_in_ms = 0;
+
+CREATE INDEX rrank ON cycling.rank_by_year_and_name (rank);
+
+CREATE INDEX ryear ON cycling.rank_by_year_and_name (race_year);
+
+CREATE CUSTOM INDEX rrank_custom ON cycling.rank_by_year_and_name (rank) USING 'org.apache.cassandra.index.CustomIndexTest';
diff --git a/cassandra-analytics-core/src/test/spark2/org/apache/cassandra/spark/bulkwriter/TestTaskContext.java b/cassandra-analytics-core/src/test/spark2/org/apache/cassandra/spark/bulkwriter/TestTaskContext.java
new file mode 100644
index 0000000..06200a2
--- /dev/null
+++ b/cassandra-analytics-core/src/test/spark2/org/apache/cassandra/spark/bulkwriter/TestTaskContext.java
@@ -0,0 +1,169 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+import java.util.Properties;
+
+import org.apache.spark.TaskContext;
+import org.apache.spark.executor.TaskMetrics;
+import org.apache.spark.memory.TaskMemoryManager;
+import org.apache.spark.metrics.source.Source;
+import org.apache.spark.shuffle.FetchFailedException;
+import org.apache.spark.util.AccumulatorV2;
+import org.apache.spark.util.TaskCompletionListener;
+import org.apache.spark.util.TaskFailureListener;
+import scala.Option;
+import scala.collection.Seq;
+
+public class TestTaskContext extends TaskContext
+{
+    @Override
+    public boolean isCompleted()
+    {
+        return false;
+    }
+
+    @Override
+    public boolean isInterrupted()
+    {
+        return false;
+    }
+
+    @Override
+    @Deprecated
+    public boolean isRunningLocally()
+    {
+        return false;
+    }
+
+    @Override
+    public TaskContext addTaskCompletionListener(TaskCompletionListener listener)
+    {
+        return null;
+    }
+
+    @Override
+    public TaskContext addTaskFailureListener(TaskFailureListener listener)
+    {
+        return null;
+    }
+
+    @Override
+    public int stageId()
+    {
+        return 0;
+    }
+
+    @Override
+    public int stageAttemptNumber()
+    {
+        return 0;
+    }
+
+    @Override
+    public int partitionId()
+    {
+        return 0;
+    }
+
+    @Override
+    public int attemptNumber()
+    {
+        return 0;
+    }
+
+    @Override
+    public long taskAttemptId()
+    {
+        return 0;
+    }
+
+    @Override
+    public String getLocalProperty(String key)
+    {
+        return null;
+    }
+
+    @Override
+    public TaskMetrics taskMetrics()
+    {
+        return null;
+    }
+
+    @Override
+    public Seq<Source> getMetricsSources(String sourceName)
+    {
+        return null;
+    }
+
+    @Override
+    public void killTaskIfInterrupted()
+    {
+    }
+
+    @Override
+    public Option<String> getKillReason()
+    {
+        return null;
+    }
+
+    @Override
+    public TaskMemoryManager taskMemoryManager()
+    {
+        return null;
+    }
+
+    @Override
+    public void registerAccumulator(AccumulatorV2 accumulator)
+    {
+    }
+
+    @Override
+    public void setFetchFailed(FetchFailedException fetchFailed)
+    {
+    }
+
+    @Override
+    public void markInterrupted(String reason)
+    {
+    }
+
+    @Override
+    public void markTaskFailed(Throwable error)
+    {
+    }
+
+    @Override
+    public void markTaskCompleted(Option<Throwable> error)
+    {
+    }
+
+    @Override
+    public Option<FetchFailedException> fetchFailed()
+    {
+        return null;
+    }
+
+    @Override
+    public Properties getLocalProperties()
+    {
+        return null;
+    }
+}
diff --git a/cassandra-analytics-core/src/test/spark2/org/apache/cassandra/spark/sparksql/PartitionKeyFilterTests.java b/cassandra-analytics-core/src/test/spark2/org/apache/cassandra/spark/sparksql/PartitionKeyFilterTests.java
new file mode 100644
index 0000000..99e0a2c
--- /dev/null
+++ b/cassandra-analytics-core/src/test/spark2/org/apache/cassandra/spark/sparksql/PartitionKeyFilterTests.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.cassandra.spark.sparksql;
+
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import com.google.common.collect.Range;
+import org.junit.Test;
+
+import org.apache.cassandra.bridge.TokenRange;
+import org.apache.cassandra.spark.TestUtils;
+import org.apache.cassandra.spark.data.partitioner.CassandraRing;
+import org.apache.cassandra.spark.data.partitioner.Partitioner;
+import org.apache.cassandra.spark.data.partitioner.TokenPartitioner;
+import org.apache.cassandra.spark.reader.SparkSSTableReader;
+import org.apache.cassandra.spark.sparksql.filters.PartitionKeyFilter;
+import org.apache.cassandra.spark.utils.RangeUtils;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+import static org.quicktheories.QuickTheory.qt;
+import static org.quicktheories.generators.SourceDSL.arbitrary;
+
+public class PartitionKeyFilterTests
+{
+    @Test
+    public void testValidFilter()
+    {
+        qt().forAll(TestUtils.bridges())
+            .checkAssert(bridge -> {
+                ByteBuffer key = bridge.aInt().serialize(10);
+                BigInteger token = bridge.hash(Partitioner.Murmur3Partitioner, key);
+                PartitionKeyFilter filter = PartitionKeyFilter.create(key, token);
+
+                ByteBuffer diffKey = bridge.aInt().serialize(11);
+                TokenRange inRange = TokenRange.singleton(token);
+                TokenRange notInRange = TokenRange.singleton(token.subtract(BigInteger.ONE));
+                SparkSSTableReader reader = mock(SparkSSTableReader.class);
+                when(reader.range()).thenReturn(TokenRange.singleton(token));
+
+                assertTrue(filter.filter(key));
+                assertFalse(filter.filter(diffKey));
+                assertTrue(filter.overlaps(inRange));
+                assertFalse(filter.overlaps(notInRange));
+                assertTrue(filter.matches(key));
+                assertFalse(filter.matches(diffKey));
+                assertTrue(SparkSSTableReader.overlaps(reader, filter.tokenRange()));
+            });
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void testEmptyKey()
+    {
+        PartitionKeyFilter.create(ByteBuffer.wrap(new byte[0]), BigInteger.ZERO);
+    }
+
+    @Test
+    public void testTokenRing()
+    {
+        qt().forAll(TestUtils.bridges(), TestUtils.partitioners(), arbitrary().pick(Arrays.asList(1, 3, 6, 12, 128)))
+            .checkAssert((bridge, partitioner, numInstances) -> {
+                CassandraRing ring = TestUtils.createRing(partitioner, numInstances);
+                TokenPartitioner tokenPartitioner = new TokenPartitioner(ring, 24, 24);
+                List<BigInteger> boundaryTokens = IntStream.range(0, tokenPartitioner.numPartitions())
+                                                           .mapToObj(tokenPartitioner::getTokenRange)
+                                                           .map(range -> Arrays.asList(range.lowerEndpoint(),
+                                                                                       midPoint(range),
+                                                                                       range.upperEndpoint()))
+                                                           .flatMap(Collection::stream)
+                                                           .collect(Collectors.toList());
+                for (BigInteger token : boundaryTokens)
+                {
+                    // Check boundary tokens only match 1 Spark token range
+                    PartitionKeyFilter filter = PartitionKeyFilter.create(bridge.aInt().serialize(11), token);
+                    assertEquals(1, tokenPartitioner.subRanges().stream()
+                                                                .map(RangeUtils::toTokenRange)
+                                                                .filter(filter::overlaps)
+                                                                .count());
+                }
+            });
+    }
+
+    private static BigInteger midPoint(Range<BigInteger> range)
+    {
+        return range.upperEndpoint()
+                    .subtract(range.lowerEndpoint())
+                    .divide(BigInteger.valueOf(2L))
+                    .add(range.lowerEndpoint());
+    }
+}
diff --git a/cassandra-analytics-core/src/test/spark2/org/apache/cassandra/spark/sparksql/SparkRangeFilterTests.java b/cassandra-analytics-core/src/test/spark2/org/apache/cassandra/spark/sparksql/SparkRangeFilterTests.java
new file mode 100644
index 0000000..6ddc465
--- /dev/null
+++ b/cassandra-analytics-core/src/test/spark2/org/apache/cassandra/spark/sparksql/SparkRangeFilterTests.java
@@ -0,0 +1,53 @@
+/*
+ * 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.cassandra.spark.sparksql;
+
+import java.math.BigInteger;
+
+import org.junit.Test;
+
+import org.apache.cassandra.bridge.TokenRange;
+import org.apache.cassandra.spark.reader.SparkSSTableReader;
+import org.apache.cassandra.spark.sparksql.filters.SparkRangeFilter;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class SparkRangeFilterTests
+{
+    @Test
+    public void testValidFilter()
+    {
+        TokenRange connected = TokenRange.closed(BigInteger.ONE, BigInteger.valueOf(2L));
+        TokenRange notConnected = TokenRange.closed(BigInteger.valueOf(2L), BigInteger.TEN);
+
+        SparkRangeFilter filter = SparkRangeFilter.create(TokenRange.closed(BigInteger.ZERO, BigInteger.ONE));
+        SparkSSTableReader reader = mock(SparkSSTableReader.class);
+        when(reader.range()).thenReturn(connected);
+
+        assertTrue(filter.overlaps(connected));
+        assertFalse(filter.overlaps(notConnected));
+        assertTrue(filter.skipPartition(BigInteger.TEN));
+        assertFalse(filter.skipPartition(BigInteger.ONE));
+        assertTrue(SparkSSTableReader.overlaps(reader, filter.tokenRange()));
+    }
+}
diff --git a/cassandra-analytics-core/src/test/spark2/org/apache/cassandra/spark/sparksql/SparkRowIteratorTests.java b/cassandra-analytics-core/src/test/spark2/org/apache/cassandra/spark/sparksql/SparkRowIteratorTests.java
new file mode 100644
index 0000000..024a882
--- /dev/null
+++ b/cassandra-analytics-core/src/test/spark2/org/apache/cassandra/spark/sparksql/SparkRowIteratorTests.java
@@ -0,0 +1,274 @@
+/*
+ * 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.cassandra.spark.sparksql;
+
+import java.io.IOException;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+
+import org.junit.Test;
+
+import org.apache.cassandra.bridge.CassandraBridge;
+import org.apache.cassandra.bridge.CassandraBridgeFactory;
+import org.apache.cassandra.bridge.CassandraVersion;
+import org.apache.cassandra.spark.TestUtils;
+import org.apache.cassandra.spark.data.CqlField;
+import org.apache.cassandra.spark.data.CqlTable;
+import org.apache.cassandra.spark.data.DataLayer;
+import org.apache.cassandra.spark.data.VersionRunner;
+import org.apache.cassandra.spark.reader.Rid;
+import org.apache.cassandra.spark.reader.StreamScanner;
+import org.apache.cassandra.spark.sparksql.filters.PartitionKeyFilter;
+import org.apache.cassandra.spark.stats.Stats;
+import org.apache.cassandra.spark.utils.ColumnTypes;
+import org.apache.cassandra.spark.utils.test.TestSchema;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.ArgumentMatchers.anyInt;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyListOf;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+import static org.quicktheories.QuickTheory.qt;
+
+public class SparkRowIteratorTests extends VersionRunner
+{
+    private static final int NUM_ROWS = 50;
+
+    public SparkRowIteratorTests(CassandraVersion version)
+    {
+        super(version);
+    }
+
+    @Test
+    public void testBasicKeyValue()
+    {
+        // I.e. "create table keyspace.table (a %s, b %s, primary key(a));"
+        qt().forAll(TestUtils.versions(), TestUtils.cql3Type(bridge), TestUtils.cql3Type(bridge))
+            .checkAssert((version, type1, type2) -> runTest(version, TestSchema.builder()
+                    .withPartitionKey("a", type1)
+                    .withColumn("b", type2)
+                    .build()));
+    }
+
+    @Test
+    public void testMultiPartitionKeys()
+    {
+        qt().forAll(TestUtils.versions(), TestUtils.cql3Type(bridge), TestUtils.cql3Type(bridge), TestUtils.cql3Type(bridge))
+            .checkAssert((version, type1, type2, type3) -> runTest(version, TestSchema.builder()
+                    .withPartitionKey("a", type1)
+                    .withPartitionKey("b", type2)
+                    .withPartitionKey("c", type3)
+                    .withColumn("d", bridge.bigint())
+                    .build()));
+    }
+
+    @Test
+    public void testBasicClusteringKey()
+    {
+        for (CassandraVersion version : TestUtils.testableVersions())
+        {
+            qt().forAll(TestUtils.cql3Type(bridge), TestUtils.cql3Type(bridge), TestUtils.cql3Type(bridge), TestUtils.sortOrder())
+                .checkAssert((type1, type2, type3, order) -> runTest(version, TestSchema.builder()
+                        .withPartitionKey("a", type1)
+                        .withClusteringKey("b", type2)
+                        .withColumn("c", type3)
+                        .withSortOrder(order)
+                        .build()));
+        }
+    }
+
+    @Test
+    public void testMultiClusteringKey()
+    {
+        for (CassandraVersion version : TestUtils.testableVersions())
+        {
+            qt().forAll(TestUtils.cql3Type(bridge), TestUtils.cql3Type(bridge), TestUtils.sortOrder(), TestUtils.sortOrder())
+                .checkAssert((type1, type2, order1, order2) -> runTest(version, TestSchema.builder()
+                        .withPartitionKey("a", bridge.bigint())
+                        .withClusteringKey("b", type1)
+                        .withClusteringKey("c", type2)
+                        .withColumn("d", bridge.bigint())
+                        .withSortOrder(order1)
+                        .withSortOrder(order2)
+                        .build()));
+        }
+    }
+
+    @Test
+    public void testUdt()
+    {
+        qt().forAll(TestUtils.cql3Type(bridge), TestUtils.cql3Type(bridge))
+            .checkAssert((type1, type2) -> runTest(CassandraVersion.FOURZERO, TestSchema.builder()
+                    .withPartitionKey("a", bridge.bigint())
+                    .withClusteringKey("b", bridge.text())
+                    .withColumn("c", bridge.udt("keyspace", "testudt")
+                                           .withField("x", type1)
+                                           .withField("y", bridge.ascii())
+                                           .withField("z", type2)
+                                           .build())
+                    .build()));
+    }
+
+    @Test
+    public void testTuple()
+    {
+        qt().forAll(TestUtils.cql3Type(bridge), TestUtils.cql3Type(bridge))
+            .checkAssert((type1, type2) -> runTest(CassandraVersion.FOURZERO, TestSchema.builder()
+                    .withPartitionKey("a", bridge.bigint())
+                    .withClusteringKey("b", bridge.text())
+                    .withColumn("c", bridge.tuple(bridge.aInt(), type1, bridge.ascii(), type2, bridge.date()))
+                    .build()));
+    }
+
+    private static void runTest(CassandraVersion version, TestSchema schema)
+    {
+        runTest(version, schema, schema.randomRows(NUM_ROWS));
+    }
+
+    private static void runTest(CassandraVersion version, TestSchema schema, TestSchema.TestRow[] testRows)
+    {
+        try
+        {
+            schema.setCassandraVersion(version);
+            testRowIterator(version, schema, testRows);
+        }
+        catch (IOException exception)
+        {
+            throw new RuntimeException(exception);
+        }
+    }
+
+    private static void testRowIterator(CassandraVersion version,
+                                        TestSchema schema,
+                                        TestSchema.TestRow[] testRows) throws IOException
+    {
+        CassandraBridge bridge = CassandraBridgeFactory.get(version);
+        CqlTable cqlTable = schema.buildTable();
+        int numRows = testRows.length;
+        int numColumns = cqlTable.fields().size() - cqlTable.numPartitionKeys() - cqlTable.numClusteringKeys();
+        List<CqlField> columns = cqlTable.fields().stream()
+                                                  .filter(field -> !field.isPartitionKey())
+                                                  .filter(field -> !field.isClusteringColumn())
+                                                  .sorted()
+                                                  .collect(Collectors.toList());
+        Rid rid = new Rid();
+        AtomicInteger rowPos = new AtomicInteger();
+        AtomicInteger colPos = new AtomicInteger();
+
+        // Mock data layer
+        DataLayer dataLayer = mock(DataLayer.class);
+        when(dataLayer.cqlTable()).thenReturn(cqlTable);
+        when(dataLayer.version()).thenCallRealMethod();
+        when(dataLayer.isInPartition(anyInt(), any(BigInteger.class), any(ByteBuffer.class))).thenReturn(true);
+        when(dataLayer.bridge()).thenReturn(bridge);
+        when(dataLayer.stats()).thenReturn(Stats.DoNothingStats.INSTANCE);
+        when(dataLayer.requestedFeatures()).thenCallRealMethod();
+
+        // Mock scanner
+        StreamScanner scanner = mock(StreamScanner.class);
+        when(scanner.rid()).thenReturn(rid);
+        doAnswer(invocation -> {
+            int col = colPos.getAndIncrement();
+            if (rowPos.get() >= numRows)
+            {
+                return false;
+            }
+            TestSchema.TestRow testRow = testRows[rowPos.get()];
+            // Write next partition key
+            if (col == 0)
+            {
+                if (cqlTable.numPartitionKeys() == 1)
+                {
+                    CqlField partitionKey = cqlTable.partitionKeys().get(0);
+                    rid.setPartitionKeyCopy(partitionKey.serialize(testRow.get(partitionKey.position())), BigInteger.ONE);
+                }
+                else
+                {
+                    assert cqlTable.numPartitionKeys() > 1;
+                    ByteBuffer[] partitionBuffers = new ByteBuffer[cqlTable.numPartitionKeys()];
+                    int position = 0;
+                    for (CqlField partitionKey : cqlTable.partitionKeys())
+                    {
+                        partitionBuffers[position] = partitionKey.serialize(testRow.get(partitionKey.position()));
+                        position++;
+                    }
+                    rid.setPartitionKeyCopy(ColumnTypes.build(false, partitionBuffers), BigInteger.ONE);
+                }
+            }
+
+            // Write next clustering keys & column name
+            CqlField column = columns.get(col);
+            ByteBuffer[] colBuffers = new ByteBuffer[cqlTable.numClusteringKeys() + 1];
+            int position = 0;
+            for (CqlField clusteringColumn : cqlTable.clusteringKeys())
+            {
+                colBuffers[position] = clusteringColumn.serialize(testRow.get(clusteringColumn.position()));
+                position++;
+            }
+            colBuffers[position] = bridge.ascii().serialize(column.name());
+            rid.setColumnNameCopy(ColumnTypes.build(false, colBuffers));
+
+            // Write value, timestamp and tombstone
+            rid.setValueCopy(column.serialize(testRow.get(column.position())));
+
+            // Move to next row
+            if (colPos.get() == numColumns)
+            {
+                if (rowPos.getAndIncrement() >= numRows)
+                {
+                    throw new IllegalStateException("Went too far...");
+                }
+                // Reset column position
+                colPos.set(0);
+            }
+
+            return true;
+        }).when(scanner).hasNext();
+
+        when(dataLayer.openCompactionScanner(anyInt(), anyListOf(PartitionKeyFilter.class), any())).thenReturn(scanner);
+
+        // Use SparkRowIterator and verify values match expected
+        SparkRowIterator it = new SparkRowIterator(0, dataLayer);
+        int rowCount = 0;
+        while (it.next())
+        {
+            while (rowCount < testRows.length && testRows[rowCount].isTombstone())
+            // Skip tombstones
+            {
+                rowCount++;
+            }
+            if (rowCount >= testRows.length)
+            {
+                break;
+            }
+
+            TestSchema.TestRow row = testRows[rowCount];
+            assertEquals(row, schema.toTestRow(it.get()));
+            rowCount++;
+        }
+        assertEquals(numRows, rowCount);
+        it.close();
+    }
+}
diff --git a/cassandra-analytics-core/src/test/spark3/org/apache/cassandra/spark/bulkwriter/TestTaskContext.java b/cassandra-analytics-core/src/test/spark3/org/apache/cassandra/spark/bulkwriter/TestTaskContext.java
new file mode 100644
index 0000000..c0ebca3
--- /dev/null
+++ b/cassandra-analytics-core/src/test/spark3/org/apache/cassandra/spark/bulkwriter/TestTaskContext.java
@@ -0,0 +1,180 @@
+/*
+ * 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.cassandra.spark.bulkwriter;
+
+import java.util.Properties;
+
+import org.apache.spark.TaskContext;
+import org.apache.spark.executor.TaskMetrics;
+import org.apache.spark.memory.TaskMemoryManager;
+import org.apache.spark.metrics.source.Source;
+import org.apache.spark.resource.ResourceInformation;
+import org.apache.spark.shuffle.FetchFailedException;
+import org.apache.spark.util.AccumulatorV2;
+import org.apache.spark.util.TaskCompletionListener;
+import org.apache.spark.util.TaskFailureListener;
+import scala.Option;
+import scala.collection.immutable.Map;
+import scala.collection.immutable.Seq;
+
+public class TestTaskContext extends TaskContext
+{
+    @Override
+    public boolean isCompleted()
+    {
+        return false;
+    }
+
+    @Override
+    public boolean isInterrupted()
+    {
+        return false;
+    }
+
+    @Deprecated
+    public boolean isRunningLocally()
+    {
+        return false;
+    }
+
+    @Override
+    public TaskContext addTaskCompletionListener(TaskCompletionListener listener)
+    {
+        return null;
+    }
+
+    @Override
+    public TaskContext addTaskFailureListener(TaskFailureListener listener)
+    {
+        return null;
+    }
+
+    @Override
+    public int stageId()
+    {
+        return 0;
+    }
+
+    @Override
+    public int stageAttemptNumber()
+    {
+        return 0;
+    }
+
+    @Override
+    public int partitionId()
+    {
+        return 0;
+    }
+
+    @Override
+    public int attemptNumber()
+    {
+        return 0;
+    }
+
+    @Override
+    public long taskAttemptId()
+    {
+        return 0;
+    }
+
+    @Override
+    public String getLocalProperty(String key)
+    {
+        return null;
+    }
+
+    public Map<String, ResourceInformation> resources()
+    {
+        return null;
+    }
+
+    public java.util.Map<String, ResourceInformation> resourcesJMap()
+    {
+        return null;
+    }
+
+    @Override
+    public TaskMetrics taskMetrics()
+    {
+        return null;
+    }
+
+    @Override
+    public Seq<Source> getMetricsSources(String sourceName)
+    {
+        return null;
+    }
+
+    @Override
+    public void killTaskIfInterrupted()
+    {
+    }
+
+    @Override
+    public Option<String> getKillReason()
+    {
+        return null;
+    }
+
+    @Override
+    public TaskMemoryManager taskMemoryManager()
+    {
+        return null;
+    }
+
+    @Override
+    public void registerAccumulator(AccumulatorV2 accumulator)
+    {
+    }
+
+    @Override
+    public void setFetchFailed(FetchFailedException fetchFailed)
+    {
+    }
+
+    @Override
+    public void markInterrupted(String reason)
+    {
+    }
+
+    @Override
+    public void markTaskFailed(Throwable error)
+    {
+    }
+
+    @Override
+    public void markTaskCompleted(Option<Throwable> error)
+    {
+    }
+
+    @Override
+    public Option<FetchFailedException> fetchFailed()
+    {
+        return null;
+    }
+
+    @Override
+    public Properties getLocalProperties()
+    {
+        return null;
+    }
+}
diff --git a/cassandra-analytics-core/src/test/spark3/org/apache/cassandra/spark/cdc/CdcTester.java b/cassandra-analytics-core/src/test/spark3/org/apache/cassandra/spark/cdc/CdcTester.java
new file mode 100644
index 0000000..076976e
--- /dev/null
+++ b/cassandra-analytics-core/src/test/spark3/org/apache/cassandra/spark/cdc/CdcTester.java
@@ -0,0 +1,402 @@
+/*
+ * 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.cassandra.spark.cdc;
+
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import com.google.common.util.concurrent.Uninterruptibles;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.bridge.CassandraBridge;
+import org.apache.cassandra.spark.TestUtils;
+import org.apache.cassandra.spark.Tester;
+import org.apache.cassandra.spark.data.CqlTable;
+import org.apache.cassandra.spark.data.partitioner.Partitioner;
+import org.apache.cassandra.spark.sparksql.LocalDataSource;
+import org.apache.cassandra.spark.utils.IOUtils;
+import org.apache.cassandra.spark.utils.ThrowableUtils;
+import org.apache.cassandra.spark.utils.test.TestSchema;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.streaming.StreamingQuery;
+import org.apache.spark.sql.streaming.StreamingQueryException;
+import org.jetbrains.annotations.Nullable;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.fail;
+
+/**
+ * Helper for writing CommitLogs using the TestSchema
+ * and reading back with Spark Streaming to verify matches the expected
+ */
+public class CdcTester
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(CdcTester.class);
+    public static final int DEFAULT_NUM_ROWS = 1000;
+
+    // TODO: Use generic CommitLog
+    public static CassandraBridge.ICommitLog COMMIT_LOG;  // CHECKSTYLE IGNORE: Constant cannot be made final
+
+    public static void setup(CassandraBridge bridge, TemporaryFolder testFolder)
+    {
+        bridge.setCommitLogPath(testFolder.getRoot().toPath());
+        bridge.setCDC(testFolder.getRoot().toPath());
+        COMMIT_LOG = bridge.testCommitLog(testFolder.getRoot());
+    }
+
+    public static void tearDown()
+    {
+        if (COMMIT_LOG != null)
+        {
+            try
+            {
+                COMMIT_LOG.stop();
+            }
+            finally
+            {
+                COMMIT_LOG.clear();
+            }
+        }
+    }
+
+    public void reset()
+    {
+        LOGGER.info("Resetting CDC test environment testId={} schema='{}' thread={}",
+                    testId, cqlTable.fields(), Thread.currentThread().getName());
+        IOUtils.clearDirectory(outputDir, path -> LOGGER.info("Clearing test output path={}", path.toString()));
+        CdcTester.tearDown();
+        COMMIT_LOG.start();
+    }
+
+    final CassandraBridge bridge;
+    @Nullable
+    final Set<String> requiredColumns;
+    final UUID testId;
+    final Path testDir;
+    final Path outputDir;
+    final Path checkpointDir;
+    final TestSchema schema;
+    final CqlTable cqlTable;
+    final int numRows;
+    final int expectedNumRows;
+    final List<CdcWriter> writers;
+    int count = 0;
+    final String dataSourceFQCN;
+    final boolean addLastModificationTime;
+    BiConsumer<Map<String, TestSchema.TestRow>, List<Row>> rowChecker;
+    BiConsumer<Map<String, TestSchema.TestRow>, List<TestSchema.TestRow>> checker;
+
+    // CHECKSTYLE IGNORE: Constructor with many parameters
+    CdcTester(CassandraBridge bridge,
+              TestSchema schema,
+              Path testDir,
+              List<CdcWriter> writers,
+              int numRows,
+              int expectedNumRows,
+              String dataSourceFQCN,
+              boolean addLastModificationTime,
+              BiConsumer<Map<String, TestSchema.TestRow>, List<Row>> rowChecker,
+              BiConsumer<Map<String, TestSchema.TestRow>, List<TestSchema.TestRow>> checker)
+    {
+        this.bridge = bridge;
+        this.testId = UUID.randomUUID();
+        this.testDir = testDir;
+        this.writers = writers;
+        this.outputDir = testDir.resolve(testId + "_out");
+        this.checkpointDir = testDir.resolve(testId + "_checkpoint");
+        this.requiredColumns = null;
+        this.numRows = numRows;
+        this.expectedNumRows = expectedNumRows;
+        this.dataSourceFQCN = dataSourceFQCN;
+        this.addLastModificationTime = addLastModificationTime;
+        this.rowChecker = rowChecker;
+        this.checker = checker;
+        try
+        {
+            Files.createDirectory(outputDir);
+        }
+        catch (IOException exception)
+        {
+            throw new RuntimeException(exception);
+        }
+
+        schema.setCassandraVersion(bridge.getVersion());
+        this.schema = schema;
+        this.cqlTable = schema.buildTable();
+    }
+
+    public interface CdcWriter
+    {
+        void write(CdcTester tester, Map<String, TestSchema.TestRow> rows, BiConsumer<TestSchema.TestRow, Long> writer);
+    }
+
+    public static class Builder
+    {
+        CassandraBridge bridge;
+        TestSchema.Builder schemaBuilder;
+        Path testDir;
+        int numRows = CdcTester.DEFAULT_NUM_ROWS;
+        int expecetedNumRows = numRows;
+        List<CdcWriter> writers = new ArrayList<>();
+        String dataSourceFQCN = LocalDataSource.class.getName();
+        boolean addLastModificationTime = false;
+        BiConsumer<Map<String, TestSchema.TestRow>, List<TestSchema.TestRow>> checker;
+        BiConsumer<Map<String, TestSchema.TestRow>, List<Row>> rowChecker;
+
+        Builder(CassandraBridge bridge, TestSchema.Builder schemaBuilder, Path testDir)
+        {
+            this.bridge = bridge;
+            this.schemaBuilder = schemaBuilder;
+            this.testDir = testDir;
+
+            // Add default writer
+            this.writers.add((tester, rows, writer) -> {
+                long timestampMicros = TimeUnit.MILLISECONDS.toMicros(System.currentTimeMillis());
+                IntStream.range(0, tester.numRows)
+                         .forEach(row -> writer.accept(Tester.newUniqueRow(tester.schema, rows), timestampMicros));
+            });
+        }
+
+        Builder clearWriters()
+        {
+            writers.clear();
+            return this;
+        }
+
+        Builder withWriter(CdcWriter writer)
+        {
+            writers.add(writer);
+            return this;
+        }
+
+        Builder withNumRows(int numRows)
+        {
+            this.numRows = numRows;
+            return this;
+        }
+
+        Builder withExpectedNumRows(int expectedNumRows)
+        {
+            this.expecetedNumRows = expectedNumRows;
+            return this;
+        }
+
+        Builder withDataSource(String dataSourceFQCN)
+        {
+            this.dataSourceFQCN = dataSourceFQCN;
+            return this;
+        }
+
+        Builder withAddLastModificationTime(boolean addLastModificationTime)
+        {
+            this.addLastModificationTime = addLastModificationTime;
+            return this;
+        }
+
+        Builder withRowChecker(Consumer<List<Row>> rowChecker)
+        {
+            this.rowChecker = (idnored, rows) -> rowChecker.accept(rows);
+            return this;
+        }
+
+        Builder withSparkRowTestRowsChecker(BiConsumer<Map<String, TestSchema.TestRow>, List<Row>> rowChecker)
+        {
+            this.rowChecker = rowChecker;
+            return this;
+        }
+
+        Builder withChecker(BiConsumer<Map<String, TestSchema.TestRow>, List<TestSchema.TestRow>> checker)
+        {
+            this.checker = checker;
+            return this;
+        }
+
+        void run()
+        {
+            new CdcTester(bridge,
+                          schemaBuilder.build(),
+                          testDir,
+                          writers,
+                          numRows,
+                          expecetedNumRows,
+                          dataSourceFQCN,
+                          addLastModificationTime,
+                          rowChecker,
+                          checker).run();
+        }
+    }
+
+    void logRow(TestSchema.TestRow row, long timestamp)
+    {
+        bridge.log(cqlTable, COMMIT_LOG, row, timestamp);
+        count++;
+    }
+
+    void run()
+    {
+        Map<String, TestSchema.TestRow> rows = new LinkedHashMap<>(numRows);
+        List<TestSchema.TestRow> actualRows = Collections.emptyList();
+        List<Row> rowsRead = null;
+
+        try
+        {
+            LOGGER.info("Running CDC test testId={} schema='{}' thread={}",
+                        testId, cqlTable.fields(), Thread.currentThread().getName());
+            CqlTable cqlTable = bridge.buildSchema(schema.createStatement, schema.keyspace);
+
+            // Write some mutations to CDC CommitLog
+            for (CdcWriter writer : writers)
+            {
+                writer.write(this, rows, (row, timestamp) -> {
+                    rows.put(row.getKey(), row);
+                    logRow(row, timestamp);
+                });
+            }
+            COMMIT_LOG.sync();
+            LOGGER.info("Logged mutations={} testId={}", count, testId);
+
+            // Run streaming query and output to outputDir
+            StreamingQuery query = TestUtils.openStreaming(cqlTable.keyspace(),
+                                                           cqlTable.createStatement(),
+                                                           bridge.getVersion(),
+                                                           Partitioner.Murmur3Partitioner,
+                                                           testDir.resolve("cdc"),
+                                                           outputDir,
+                                                           checkpointDir,
+                                                           dataSourceFQCN,
+                                                           addLastModificationTime);
+            // Wait for query to write output parquet files before reading to verify test output matches expected
+            int prevNumRows = 0;
+            long timeout = System.nanoTime();
+            while (actualRows.size() < expectedNumRows)
+            {
+                rowsRead = readRows();
+                actualRows = toTestRows(rowsRead);
+                timeout = prevNumRows == actualRows.size() ? timeout : System.nanoTime();
+                prevNumRows = actualRows.size();
+                long seconds = TimeUnit.NANOSECONDS.toSeconds(System.nanoTime() - timeout);
+                if (seconds > 30)
+                {
+                    // Timeout eventually if no progress
+                    LOGGER.warn("Expected {} rows only {} found after {} seconds testId={} ",
+                                expectedNumRows, prevNumRows, seconds, testId);
+                    break;
+                }
+                Uninterruptibles.sleepUninterruptibly(500, TimeUnit.MILLISECONDS);
+            }
+
+            query.stop();
+            query.awaitTermination();
+        }
+        catch (StreamingQueryException exception)
+        {
+            if (!exception.getCause().getMessage().startsWith("Job aborted"))
+            {
+                fail("SparkStreaming job failed with exception: " + exception.getMessage());
+            }
+        }
+        catch (TimeoutException exception)
+        {
+            fail("Streaming query timed out: " + exception.getMessage());
+        }
+        catch (Throwable throwable)
+        {
+            LOGGER.error("Unexpected error in CdcTester", ThrowableUtils.rootCause(throwable));
+        }
+        finally
+        {
+            try
+            {
+                // Read streaming output from outputDir and verify the rows match expected
+                LOGGER.info("Finished CDC test, verifying output testId={} schema='{}' thread={} actualRows={}",
+                            testId, cqlTable.fields(), Thread.currentThread().getName(), actualRows.size());
+
+                if (rowChecker != null)
+                {
+                    assertNotNull(rowsRead);
+                    rowChecker.accept(rows, rowsRead);
+                }
+
+                if (checker == null)
+                {
+                    int actualRowCount = 0;
+                    for (TestSchema.TestRow actualRow : actualRows)
+                    {
+                        String key = actualRow.getKey();
+                        TestSchema.TestRow expectedRow = rows.get(key);
+                        assertNotNull(expectedRow);
+                        assertEquals("Row read in Spark does not match expected",
+                                     expectedRow.withColumns(requiredColumns).nullifyUnsetColumn(), actualRow);
+                        actualRowCount++;
+                    }
+                    assertEquals(String.format("Expected %d rows, but %d read testId=%s",
+                                               expectedNumRows, actualRowCount, testId), rows.size(), actualRowCount);
+                }
+                else
+                {
+                    checker.accept(rows, actualRows);
+                }
+            }
+            finally
+            {
+                reset();
+            }
+        }
+    }
+
+    private List<Row> readRows()
+    {
+        return TestUtils.read(outputDir, TestSchema.toStructType(cqlTable, addLastModificationTime)).collectAsList();
+    }
+
+    private List<TestSchema.TestRow> toTestRows(List<Row> rows)
+    {
+        return rows.stream()
+                   .map(row -> schema.toTestRow(row, requiredColumns))
+                   .collect(Collectors.toList());
+    }
+
+    public static Builder builder(CassandraBridge bridge, TemporaryFolder testDir, TestSchema.Builder schemaBuilder)
+    {
+        return builder(bridge, testDir.getRoot().toPath(), schemaBuilder);
+    }
+
+    public static Builder builder(CassandraBridge bridge, Path testDir, TestSchema.Builder schemaBuilder)
+    {
+        return new Builder(bridge, schemaBuilder, testDir);
+    }
+}
diff --git a/cassandra-analytics-core/src/test/spark3/org/apache/cassandra/spark/cdc/CdcTests.java b/cassandra-analytics-core/src/test/spark3/org/apache/cassandra/spark/cdc/CdcTests.java
new file mode 100644
index 0000000..9fcd6b1
--- /dev/null
+++ b/cassandra-analytics-core/src/test/spark3/org/apache/cassandra/spark/cdc/CdcTests.java
@@ -0,0 +1,547 @@
+/*
+ * 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.cassandra.spark.cdc;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import org.apache.cassandra.bridge.CassandraBridge;
+import org.apache.cassandra.bridge.CassandraVersion;
+import org.apache.cassandra.spark.TestUtils;
+import org.apache.cassandra.spark.Tester;
+import org.apache.cassandra.spark.cdc.watermarker.Watermarker;
+import org.apache.cassandra.spark.data.CqlField;
+import org.apache.cassandra.spark.data.CqlTable;
+import org.apache.cassandra.spark.data.LocalCommitLog;
+import org.apache.cassandra.spark.data.VersionRunner;
+import org.apache.cassandra.spark.data.partitioner.CassandraInstance;
+import org.apache.cassandra.spark.utils.test.TestSchema;
+import org.apache.spark.sql.Row;
+import org.jetbrains.annotations.Nullable;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.quicktheories.QuickTheory.qt;
+
+@Ignore
+public class CdcTests extends VersionRunner
+{
+    @ClassRule
+    public static TemporaryFolder DIRECTORY = new TemporaryFolder();  // CHECKSTYLE IGNORE: Constant cannot be made final
+
+    @Before
+    public void setup()
+    {
+        CdcTester.setup(bridge, DIRECTORY);
+    }
+
+    @After
+    public void tearDown()
+    {
+        CdcTester.tearDown();
+    }
+
+    public CdcTests(CassandraVersion version)
+    {
+        super(version);
+    }
+
+    @Test
+    public void testSinglePartitionKey()
+    {
+        qt().forAll(TestUtils.cql3Type(bridge))
+            .checkAssert(type ->
+                         CdcTester.builder(bridge, DIRECTORY, TestSchema.builder()
+                                                                        .withPartitionKey("pk", bridge.uuid())
+                                                                        .withColumn("c1", bridge.bigint())
+                                                                        .withColumn("c2", type))
+                                  .withRowChecker(sparkRows -> {
+                                      for (Row row : sparkRows)
+                                      {
+                                          byte[] updatedFieldsIndicator = (byte[]) row.get(3);
+                                          BitSet actual = BitSet.valueOf(updatedFieldsIndicator);
+                                          BitSet expected = new BitSet(3);
+                                          expected.set(0, 3);  // Expecting all columns to be set
+                                          assertEquals(expected, actual);
+                                      }
+                                  })
+                                  .run());
+    }
+
+    @Test
+    public void testUpdatedFieldsIndicator()
+    {
+        qt().forAll(TestUtils.cql3Type(bridge))
+            .checkAssert(type ->
+                         CdcTester.builder(bridge, DIRECTORY, TestSchema.builder()
+                                                                        .withPartitionKey("pk", bridge.uuid())
+                                                                        .withColumn("c1", bridge.bigint())
+                                                                        .withColumn("c2", type))
+                                  .clearWriters()
+                                  .withAddLastModificationTime(true)
+                                  .withWriter((tester, rows, writer) -> {
+                                      for (int row = 0; row < tester.numRows; row++)
+                                      {
+                                          TestSchema.TestRow testRow = Tester.newUniqueRow(tester.schema, rows);
+                                          testRow = testRow.copy("c1", CassandraBridge.UNSET_MARKER);  // Mark c1 as not updated / unset
+                                          writer.accept(testRow, TimeUnit.MILLISECONDS.toMicros(System.currentTimeMillis()));
+                                      }
+                                  })
+                                  .withRowChecker(sparkRows -> {
+                                      for (Row row : sparkRows)
+                                      {
+                                          byte[] updatedFieldsIndicator = (byte[]) row.get(4);
+                                          BitSet bs = BitSet.valueOf(updatedFieldsIndicator);
+                                          BitSet expected = new BitSet(3);
+                                          expected.set(0);  // Expecting pk to be set
+                                          expected.set(2);  // And c2 to be set
+                                          assertEquals(expected, bs);
+                                          assertNull("c1 should be null", row.get(1));
+                                      }
+                                  })
+                                  .run());
+    }
+
+    @Test
+    public void testMultipleWritesToSameKeyInBatch()
+    {
+        // The test writes different groups of mutations.
+        // Each group of mutations write to the same key with a different timestamp.
+        // For CDC, it only deduplicates and emits the replicated mutations, i.e. they have the same writetime.
+        qt()
+        .withUnlimitedExamples()
+        .withTestingTime(5, TimeUnit.MINUTES)
+        .forAll(TestUtils.cql3Type(bridge))
+            .checkAssert(type ->
+                         CdcTester.builder(bridge, DIRECTORY, TestSchema.builder()
+                                                                        .withPartitionKey("pk", bridge.uuid())
+                                                                        .withColumn("c1", bridge.bigint())
+                                                                        .withColumn("c2", type))
+                                  .clearWriters()
+                                  .withAddLastModificationTime(true)
+                                  .withWriter((tester, rows, writer) -> {
+                                      // Write initial values
+                                      long timestamp = TimeUnit.MILLISECONDS.toMicros(System.currentTimeMillis());
+                                      for (int row = 0; row < tester.numRows; row++)
+                                      {
+                                          writer.accept(Tester.newUniqueRow(tester.schema, rows), timestamp++);
+                                      }
+
+                                      // Overwrite with new mutations at later timestamp
+                                      for (TestSchema.TestRow row : rows.values())
+                                      {
+                                          TestSchema.TestRow newUniqueRow = Tester.newUniqueRow(tester.schema, rows);
+                                          for (CqlField field : tester.cqlTable.valueColumns())
+                                          {
+                                              // Update value columns
+                                              row = row.copy(field.position(), newUniqueRow.get(field.position()));
+                                          }
+                                          writer.accept(row, timestamp++);
+                                      }
+                                  })
+                                  .withChecker((testRows, actualRows) -> {
+                                      int partitions = testRows.size();
+                                      int mutations = actualRows.size();
+                                      assertEquals("Each PK should get 2 mutations", partitions * 2, mutations);
+                                  })
+                                  .withRowChecker(sparkRows -> {
+                                      long timestamp = -1L;
+                                      for (Row row : sparkRows)
+                                      {
+                                          if (timestamp < 0)
+                                          {
+                                              timestamp = getMicros(row.getTimestamp(3));
+                                          }
+                                          else
+                                          {
+                                              long lastTimestamp = timestamp;
+                                              timestamp = getMicros(row.getTimestamp(3));
+                                              assertTrue("Writetime should be monotonically increasing",
+                                                         lastTimestamp < timestamp);
+                                          }
+                                      }
+                                  })
+                                  .run());
+    }
+
+    private long getMicros(java.sql.Timestamp timestamp)
+    {
+        long millis = timestamp.getTime();
+        int nanos = timestamp.getNanos();
+        return TimeUnit.MILLISECONDS.toMicros(millis) + TimeUnit.NANOSECONDS.toMicros(nanos);
+    }
+
+    @Test
+    public void testCompactOnlyWithEnoughReplicas()
+    {
+        qt().forAll(TestUtils.cql3Type(bridge))
+            .checkAssert(type ->
+                         CdcTester.builder(bridge, DIRECTORY, TestSchema.builder()
+                                                                        .withPartitionKey("pk", bridge.uuid())
+                                                                        .withColumn("c1", bridge.bigint())
+                                                                        .withColumn("c2", type))
+                                  .withDataSource(RequireTwoReplicasLocalDataSource.class.getName())
+                                  .withNumRows(1000)
+                                  .withExpectedNumRows(999)  // Expect 1 less row
+                                  .withAddLastModificationTime(true)
+                                  .clearWriters()
+                                  .withWriter((tester, rows, writer) -> {
+                                      // Write initial values
+                                      long timestamp = System.currentTimeMillis();
+                                      Map<Long, TestSchema.TestRow> genRows = new HashMap<>();
+                                      IntStream.range(0, tester.numRows)
+                                               .forEach(row -> genRows.put(timestamp + row, Tester.newUniqueRow(tester.schema, rows)));
+                                      genRows.forEach((key, value) -> writer.accept(value, TimeUnit.MILLISECONDS.toMicros(key)));
+
+                                      // Write the same values again, with the first value skipped.
+                                      // All values except the first one have 2 copies.
+                                      // The test is using RequireTwoReplicasCompactionDataSource,
+                                      // so the output should not contain the first value.
+                                      for (long row = 1; row < tester.numRows; row++)
+                                      {
+                                          writer.accept(genRows.get(timestamp + row), TimeUnit.MILLISECONDS.toMicros(timestamp + row));
+                                      }
+                                  })
+                                  .withRowChecker(rows -> {
+                                      int size = rows.size();
+                                      // The timestamp column is added at column 4
+                                      int uniqueTsCount = rows.stream().map(r -> r.getTimestamp(3).getTime())
+                                                              .collect(Collectors.toSet())
+                                                              .size();
+                                      Assert.assertEquals("Output rows should have distinct lastModified timestamps", size, uniqueTsCount);
+                                  })
+                                  .withChecker((testRows, actualRows) -> {
+                                      Assert.assertEquals("There should be exact one row less in the output.",
+                                                          actualRows.size() + 1, testRows.size());
+                                      boolean allContains = true;
+                                      TestSchema.TestRow unexpectedRow = null;
+                                      for (TestSchema.TestRow row : actualRows)
+                                      {
+                                          if (!testRows.containsValue(row))
+                                          {
+                                              allContains = false;
+                                              unexpectedRow = row;
+                                              break;
+                                          }
+                                      }
+                                      if (!allContains && unexpectedRow != null)
+                                      {
+                                          Assert.fail("Found an unexpected row from the output: " + unexpectedRow);
+                                      }
+                                  })
+                                  .run());
+    }
+
+    @Test
+    public void testCompositePartitionKey()
+    {
+        qt().forAll(TestUtils.cql3Type(bridge))
+            .checkAssert(type ->
+                         CdcTester.builder(bridge, DIRECTORY, TestSchema.builder()
+                                                                        .withPartitionKey("pk1", bridge.uuid())
+                                                                        .withPartitionKey("pk2", type)
+                                                                        .withPartitionKey("pk3", bridge.timestamp())
+                                                                        .withColumn("c1", bridge.bigint())
+                                                                        .withColumn("c2", bridge.text()))
+                                  .run()
+            );
+    }
+
+    @Test
+    public void testClusteringKey()
+    {
+        qt().forAll(TestUtils.cql3Type(bridge))
+            .checkAssert(type ->
+                         CdcTester.builder(bridge, DIRECTORY, TestSchema.builder()
+                                                                        .withPartitionKey("pk", bridge.uuid())
+                                                                        .withPartitionKey("ck", type)
+                                                                        .withColumn("c1", bridge.bigint())
+                                                                        .withColumn("c2", bridge.text()))
+                                  .run()
+            );
+    }
+
+    @Test
+    public void testMultipleClusteringKeys()
+    {
+        qt().withExamples(50)
+            .forAll(TestUtils.cql3Type(bridge), TestUtils.cql3Type(bridge), TestUtils.cql3Type(bridge))
+            .checkAssert((type1, type2, type3) ->
+                         CdcTester.builder(bridge, DIRECTORY, TestSchema.builder()
+                                                                        .withPartitionKey("pk", bridge.uuid())
+                                                                        .withClusteringKey("ck1", type1)
+                                                                        .withClusteringKey("ck2", type2)
+                                                                        .withClusteringKey("ck3", type3)
+                                                                        .withColumn("c1", bridge.bigint())
+                                                                        .withColumn("c2", bridge.text()))
+                                  .run()
+            );
+    }
+
+    @Test
+    public void testSet()
+    {
+        qt().forAll(TestUtils.cql3Type(bridge))
+            .checkAssert(type ->
+                         CdcTester.builder(bridge, DIRECTORY, TestSchema.builder()
+                                                                        .withPartitionKey("pk", bridge.uuid())
+                                                                        .withColumn("c1", bridge.bigint())
+                                                                        .withColumn("c2", bridge.set(type)))
+                                  .run());
+    }
+
+    @Test
+    public void testList()
+    {
+        qt().forAll(TestUtils.cql3Type(bridge))
+            .checkAssert(type ->
+                         CdcTester.builder(bridge, DIRECTORY, TestSchema.builder()
+                                                                        .withPartitionKey("pk", bridge.uuid())
+                                                                        .withColumn("c1", bridge.bigint())
+                                                                        .withColumn("c2", bridge.list(type)))
+                                  .run());
+    }
+
+    @Test
+    public void testMap()
+    {
+        // TODO
+        qt().withExamples(1)
+            .forAll(TestUtils.cql3Type(bridge), TestUtils.cql3Type(bridge))
+            .checkAssert((type1, type2) ->
+                         CdcTester.builder(bridge, DIRECTORY, TestSchema.builder()
+                                                                        .withPartitionKey("pk", bridge.uuid())
+                                                                        .withColumn("c1", bridge.bigint())
+                                                                        .withColumn("c2", bridge.map(type1, type2)))
+                                  .run());
+    }
+
+    @Test
+    public void testUpdateFlag()
+    {
+        qt().withExamples(10)
+            .forAll(TestUtils.cql3Type(bridge))
+            .checkAssert(type ->
+                         CdcTester.builder(bridge, DIRECTORY, TestSchema.builder()
+                                                                        .withPartitionKey("pk", bridge.uuid())
+                                                                        .withColumn("c1", bridge.aInt())
+                                                                        .withColumn("c2", type))
+                                  .clearWriters()
+                                  .withNumRows(1000)
+                                  .withWriter((tester, rows, writer) -> {
+                                      int halfway = tester.numRows / 2;
+                                      for (int row = 0; row < tester.numRows; row++)
+                                      {
+                                          TestSchema.TestRow testRow = Tester.newUniqueRow(tester.schema, rows);
+                                          testRow = testRow.copy("c1", row);
+                                          if (row >= halfway)
+                                          {
+                                              testRow.fromUpdate();
+                                          }
+                                          writer.accept(testRow, TimeUnit.MILLISECONDS.toMicros(System.currentTimeMillis()));
+                                      }
+                                  })
+                                  .withRowChecker(sparkRows -> {
+                                      int length = sparkRows.size();
+                                      int halfway = length / 2;
+                                      for (Row row : sparkRows)
+                                      {
+                                          int index = row.getInt(1);
+                                          boolean isUpdate = row.getBoolean(4);
+                                          assertEquals(isUpdate, index >= halfway);
+                                      }
+                                  })
+                                  .run());
+    }
+
+    // CommitLog Reader
+
+    @Test
+    public void testReaderWatermarking() throws IOException
+    {
+        TestSchema schema = TestSchema.builder()
+                                      .withPartitionKey("pk", bridge.bigint())
+                                      .withColumn("c1", bridge.bigint())
+                                      .withColumn("c2", bridge.bigint())
+                                      .build();
+        CqlTable cqlTable = bridge.buildSchema(schema.createStatement, schema.keyspace);
+        int numRows = 1000;
+
+        // Write some rows to a CommitLog
+        Set<Long> keys = new HashSet<>(numRows);
+        for (int index = 0; index < numRows; index++)
+        {
+            TestSchema.TestRow row = schema.randomRow();
+            while (keys.contains(row.getLong("pk")))
+            {
+                row = schema.randomRow();
+            }
+            keys.add(row.getLong("pk"));
+            bridge.log(cqlTable, CdcTester.COMMIT_LOG, row, TimeUnit.MILLISECONDS.toMicros(System.currentTimeMillis()));
+        }
+        CdcTester.COMMIT_LOG.sync();
+
+        AtomicReference<CommitLog.Marker> currentMarker = new AtomicReference<>();
+        List<CommitLog.Marker> markers = Collections.synchronizedList(new ArrayList<>());
+        Watermarker watermarker = createWatermarker(currentMarker, markers);
+        File logFile = Files.list(DIRECTORY.getRoot().toPath().resolve("cdc"))
+                            .max((first, second) -> {
+                                try
+                                {
+                                    return Long.compare(Files.size(first), Files.size(second));
+                                }
+                                catch (IOException exception)
+                                {
+                                    throw new RuntimeException(exception);
+                                }
+                            })
+                            .orElseThrow(() -> new RuntimeException("No log files found"))
+                            .toFile();
+
+        // Read entire CommitLog and verify correct
+        Set<Long> allRows = readLog(cqlTable, logFile, watermarker, keys);
+        assertEquals(numRows, allRows.size());
+
+        // Re-read CommitLog from each watermark position and verify subset of partitions are read
+        int foundRows = allRows.size();
+        allRows.clear();
+        List<CommitLog.Marker> allMarkers = new ArrayList<>(markers);
+        CommitLog.Marker prevMarker = null;
+        for (CommitLog.Marker marker : allMarkers)
+        {
+            currentMarker.set(marker);
+            Set<Long> result = readLog(cqlTable, logFile, watermarker, keys);
+            assertTrue(result.size() < foundRows);
+            foundRows = result.size();
+            if (prevMarker != null)
+            {
+                assertTrue(prevMarker.compareTo(marker) < 0);
+                assertTrue(prevMarker.position() < marker.position());
+            }
+            prevMarker = marker;
+
+            if (marker.equals(allMarkers.get(allMarkers.size() - 1)))
+            {
+                // Last marker should return 0 updates and be at the end of the file
+                assertTrue(result.isEmpty());
+            }
+            else
+            {
+                assertFalse(result.isEmpty());
+            }
+        }
+    }
+
+    private Watermarker createWatermarker(AtomicReference<CommitLog.Marker> current, List<CommitLog.Marker> all)
+    {
+        return new Watermarker()
+        {
+            @Override
+            public Watermarker instance(String jobId)
+            {
+                return this;
+            }
+
+            @Override
+            public void recordReplicaCount(IPartitionUpdateWrapper update, int numReplicas)
+            {
+            }
+
+            @Override
+            public int replicaCount(IPartitionUpdateWrapper update)
+            {
+                return 0;
+            }
+
+            @Override
+            public void untrackReplicaCount(IPartitionUpdateWrapper update)
+            {
+            }
+
+            @Override
+            public boolean seenBefore(IPartitionUpdateWrapper update)
+            {
+                return false;
+            }
+
+            @Override
+            public void updateHighWaterMark(CommitLog.Marker marker)
+            {
+                all.add(marker);
+            }
+
+            @Override
+            @Nullable
+            public CommitLog.Marker highWaterMark(CassandraInstance instance)
+            {
+                CommitLog.Marker marker = current.get();
+                return marker != null ? marker : instance.zeroMarker();
+            }
+
+            @Override
+            public void persist(@Nullable Long maxAgeMicros)
+            {
+            }
+
+            @Override
+            public void clear()
+            {
+                all.clear();
+            }
+        };
+    }
+
+    private Set<Long> readLog(CqlTable table, File logFile, Watermarker watermarker, Set<Long> keys)
+    {
+        try (LocalCommitLog log = new LocalCommitLog(logFile))
+        {
+            Set<Long> result = bridge.readLog(table, log, watermarker);
+            result.forEach(key -> assertTrue("Unexpected keys have been read from the commit log", keys.contains(key)));
+            return result;
+        }
+        catch (Exception exception)
+        {
+            throw new RuntimeException(exception);
+        }
+    }
+}
diff --git a/cassandra-analytics-core/src/test/spark3/org/apache/cassandra/spark/cdc/CdcTombstoneTests.java b/cassandra-analytics-core/src/test/spark3/org/apache/cassandra/spark/cdc/CdcTombstoneTests.java
new file mode 100644
index 0000000..78cc444
--- /dev/null
+++ b/cassandra-analytics-core/src/test/spark3/org/apache/cassandra/spark/cdc/CdcTombstoneTests.java
@@ -0,0 +1,710 @@
+/*
+ * 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.cassandra.spark.cdc;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import org.apache.cassandra.bridge.CassandraBridge;
+import org.apache.cassandra.bridge.CassandraVersion;
+import org.apache.cassandra.bridge.RangeTombstone;
+import org.apache.cassandra.spark.TestUtils;
+import org.apache.cassandra.spark.Tester;
+import org.apache.cassandra.spark.data.CqlField;
+import org.apache.cassandra.spark.data.VersionRunner;
+import org.apache.cassandra.spark.utils.ComparisonUtils;
+import org.apache.cassandra.spark.utils.test.TestSchema;
+import org.apache.spark.sql.Row;
+import scala.collection.mutable.AbstractSeq;
+
+import static org.apache.cassandra.spark.utils.ScalaConversionUtils.mutableSeqAsJavaList;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.quicktheories.QuickTheory.qt;
+
+@Ignore
+public class CdcTombstoneTests extends VersionRunner
+{
+    @ClassRule
+    public static TemporaryFolder DIRECTORY = new TemporaryFolder();  // CHECKSTYLE IGNORE: Constant cannot be made final
+
+    @Before
+    public void setup()
+    {
+        CdcTester.setup(bridge, DIRECTORY);
+    }
+
+    @After
+    public void tearDown()
+    {
+        CdcTester.tearDown();
+    }
+
+    public CdcTombstoneTests(CassandraVersion version)
+    {
+        super(version);
+    }
+
+    private static void assertEqualsWithMessage(Object expected, Object actual)
+    {
+        assertTrue(String.format("Expect %s to equal to %s, but not.", expected, actual),
+                   ComparisonUtils.equals(expected, actual));
+    }
+
+    @Test
+    public void testCellDeletion()
+    {
+        // The test write cell-level tombstones, i.e. deleting one or more columns in a row, for CDC job to aggregate
+        qt().forAll(TestUtils.cql3Type(bridge))
+            .checkAssert(type -> CdcTester
+                .builder(bridge, DIRECTORY, TestSchema.builder()
+                                                      .withPartitionKey("pk", bridge.uuid())
+                                                      .withColumn("c1", bridge.bigint())
+                                                      .withColumn("c2", type)
+                                                      .withColumn("c3", bridge.list(type)))
+                .clearWriters()
+                .withWriter((tester, rows, writer) -> {
+                    for (int row = 0; row < tester.numRows; row++)
+                    {
+                        TestSchema.TestRow testRow = Tester.newUniqueRow(tester.schema, rows);
+                        testRow = testRow.copy("c1", CassandraBridge.UNSET_MARKER);  // Mark c1 as not updated / unset
+                        testRow = testRow.copy("c2", null);                          // Delete c2
+                        testRow = testRow.copy("c3", null);                          // Delete c3
+                        writer.accept(testRow, TimeUnit.MILLISECONDS.toMicros(System.currentTimeMillis()));
+                    }
+                })
+                .withRowChecker(sparkRows -> {
+                    for (Row row : sparkRows)
+                    {
+                        byte[] updatedFieldsIndicator = (byte[]) row.get(4);
+                        BitSet bs = BitSet.valueOf(updatedFieldsIndicator);
+                        BitSet expected = new BitSet(4);
+                        expected.set(0);  // Expecting pk to be set...
+                        expected.set(2);  // ... and c2 to be set
+                        expected.set(3);  // ... and c3 to be set
+                        assertEquals(expected, bs);
+                        assertNotNull("pk should not be null", row.get(0));  // pk should be set
+                        assertNull("c1 should be null", row.get(1));         // null due to unset
+                        assertNull("c2 should be null", row.get(2));         // null due to deletion
+                        assertNull("c3 should be null", row.get(3));         // null due to deletion
+                    }
+                })
+                .run());
+    }
+
+    @Test
+    public void testRowDeletionWithClusteringKeyAndStatic()
+    {
+        testRowDeletion(5,     // Number of columns
+                        true,  // Has clustering key?
+                        type -> TestSchema.builder()
+                                          .withPartitionKey("pk", bridge.uuid())
+                                          .withClusteringKey("ck", bridge.bigint())
+                                          .withStaticColumn("sc", bridge.bigint())
+                                          .withColumn("c1", type)
+                                          .withColumn("c2", bridge.bigint()));
+    }
+
+    @Test
+    public void testRowDeletionWithClusteringKeyNoStatic()
+    {
+        testRowDeletion(4,     // Number of columns
+                        true,  // Has clustering key?
+                        type -> TestSchema.builder()
+                                          .withPartitionKey("pk", bridge.uuid())
+                                          .withClusteringKey("ck", bridge.bigint())
+                                          .withColumn("c1", type)
+                                          .withColumn("c2", bridge.bigint()));
+    }
+
+    @Test
+    public void testRowDeletionSimpleSchema()
+    {
+        testRowDeletion(3,      // Number of columns
+                        false,  // Has clustering key?
+                        type -> TestSchema.builder()
+                                          .withPartitionKey("pk", bridge.uuid())
+                                          .withColumn("c1", type)
+                                          .withColumn("c2", bridge.bigint()));
+    }
+
+    private void testRowDeletion(int numOfColumns,
+                                 boolean hasClustering,
+                                 Function<CqlField.NativeType, TestSchema.Builder> schemaBuilder)
+    {
+        // The test write row-level tombstones.
+        // The expected output should include the values of all primary keys but all other columns should be null,
+        // i.e. [pk.., ck.., null..]. The bitset should indicate that only the primary keys are present.
+        // This kind of output means the entire row is deleted.
+        Set<Integer> rowDeletionIndices = new HashSet<>();
+        Random random = new Random(1);
+        long minTimestamp = System.currentTimeMillis();
+        int numRows = 1000;
+        qt().forAll(TestUtils.cql3Type(bridge))
+            .checkAssert(type -> CdcTester
+                .builder(bridge, DIRECTORY, schemaBuilder.apply(type))
+                .withAddLastModificationTime(true)
+                .clearWriters()
+                .withNumRows(numRows)
+                .withWriter((tester, rows, writer) -> {
+                    rowDeletionIndices.clear();
+                    long timestamp = minTimestamp;
+                    for (int row = 0; row < tester.numRows; row++)
+                    {
+                        TestSchema.TestRow testRow = Tester.newUniqueRow(tester.schema, rows);
+                        if (random.nextDouble() < 0.5)
+                        {
+                            testRow.delete();
+                            rowDeletionIndices.add(row);
+                        }
+                        timestamp += 1;
+                        writer.accept(testRow, TimeUnit.MILLISECONDS.toMicros(timestamp));
+                    }
+                })
+                // Disable checker on the test row. The check is done in the row checker below.
+                .withChecker((testRows, actualRows) -> { })
+                .withRowChecker(sparkRows -> {
+                    assertEquals("Unexpected number of rows in output", numRows, sparkRows.size());
+                    for (int index = 0; index < sparkRows.size(); index++)
+                    {
+                        Row row = sparkRows.get(index);
+                        long lmtInMillis = row.getTimestamp(numOfColumns).getTime();
+                        assertTrue("Last modification time should have a lower bound of " + minTimestamp,
+                                   lmtInMillis >= minTimestamp);
+                        byte[] updatedFieldsIndicator = (byte[]) row.get(numOfColumns + 1);  // Indicator column
+                        BitSet bs = BitSet.valueOf(updatedFieldsIndicator);
+                        BitSet expected = new BitSet(numOfColumns);
+                        if (rowDeletionIndices.contains(index))  // Verify row deletion
+                        {
+                            expected.set(0);    // Expecting pk...
+                            if (hasClustering)  // ... and ck to be set
+                            {
+                                expected.set(1);
+                            }
+                            assertEquals("row" + index + " should only have the primary keys to be flagged.",
+                                         expected, bs);
+                            assertNotNull("pk should not be null", row.get(0));  // pk should be set
+                            if (hasClustering)
+                            {
+                                assertNotNull("ck should not be null", row.get(1));  // ck should be set
+                            }
+                            for (int colIndex = hasClustering ? 2 : 1; colIndex < numOfColumns; colIndex++)
+                            {
+                                // null due to row deletion
+                                assertNull("None primary key columns should be null", row.get(colIndex));
+                            }
+                        }
+                        else
+                        {
+                            // Verify update
+                            for (int colIndex = 0; colIndex < numOfColumns; colIndex++)
+                            {
+                                expected.set(colIndex);
+                                assertNotNull("All column values should exist for full row update",
+                                              row.get(colIndex));
+                            }
+                            assertEquals("row" + index + " should have all columns set", expected, bs);
+                        }
+                    }
+                })
+                .run());
+    }
+
+    @Test
+    public void testPartitionDeletionWithStaticColumn()
+    {
+        testPartitionDeletion(5,     // Number of columns
+                              true,  // Has clustering key
+                              1,     // Partition key columns
+                              type -> TestSchema.builder()
+                                                .withPartitionKey("pk", bridge.uuid())
+                                                .withClusteringKey("ck", bridge.bigint())
+                                                .withStaticColumn("sc", bridge.bigint())
+                                                .withColumn("c1", type)
+                                                .withColumn("c2", bridge.bigint()));
+    }
+
+    @Test
+    public void testPartitionDeletionWithCompositePK()
+    {
+        testPartitionDeletion(5,     // Number of columns
+                              true,  // Has clustering key
+                              2,     // Partition key columns
+                              type -> TestSchema.builder()
+                                                .withPartitionKey("pk1", bridge.uuid())
+                                                .withPartitionKey("pk2", type)
+                                                .withClusteringKey("ck", bridge.bigint())
+                                                .withColumn("c1", type)
+                                                .withColumn("c2", bridge.bigint()));
+    }
+
+    @Test
+    public void testPartitionDeletionWithoutCK()
+    {
+        testPartitionDeletion(5,      // Number of columns
+                              false,  // Has clustering key
+                              3,      // Partition key columns
+                              type -> TestSchema.builder()
+                                                .withPartitionKey("pk1", bridge.uuid())
+                                                .withPartitionKey("pk2", type)
+                                                .withPartitionKey("pk3", bridge.bigint())
+                                                .withColumn("c1", type)
+                                                .withColumn("c2", bridge.bigint()));
+    }
+
+    // At most can have 1 clustering key when `hasClustering` is true
+    @SuppressWarnings("SameParameterValue")
+    private void testPartitionDeletion(int numOfColumns,
+                                       boolean hasClustering,
+                                       int partitionKeys,
+                                       Function<CqlField.NativeType, TestSchema.Builder> schemaBuilder)
+    {
+        // The test write partition-level tombstones.
+        // The expected output should include the values of all partition keys but all other columns should be null,
+        // i.e. [pk.., null..]. The bitset should indicate that only the partition keys are present.
+        // This kind of output means the entire partition is deleted.
+        Set<Integer> partitionDeletionIndices = new HashSet<>();
+        List<List<Object>> validationPk = new ArrayList<>();  // pk of the partition deletions
+        Random random = new Random(1);
+        long minTimestamp = System.currentTimeMillis();
+        int numRows = 1000;
+        qt().forAll(TestUtils.cql3Type(bridge))
+            .checkAssert(type -> CdcTester
+                .builder(bridge, DIRECTORY, schemaBuilder.apply(type))
+                .withAddLastModificationTime(true)
+                .clearWriters()
+                .withNumRows(numRows)
+                .withWriter((tester, rows, writer) -> {
+                    partitionDeletionIndices.clear();
+                    validationPk.clear();
+                    long timestamp = minTimestamp;
+                    for (int row = 0; row < tester.numRows; row++)
+                    {
+                        TestSchema.TestRow testRow;
+                        if (random.nextDouble() < 0.5)
+                        {
+                            testRow = Tester.newUniquePartitionDeletion(tester.schema, rows);
+                            List<Object> pk = new ArrayList<>(partitionKeys);
+                            for (int key = 0; key < partitionKeys; key++)
+                            {
+                                pk.add(testRow.get(key));
+                            }
+                            validationPk.add(pk);
+                            partitionDeletionIndices.add(row);
+                        }
+                        else
+                        {
+                            testRow = Tester.newUniqueRow(tester.schema, rows);
+                        }
+                        timestamp += 1;
+                        writer.accept(testRow, TimeUnit.MILLISECONDS.toMicros(timestamp));
+                    }
+                })
+                // Disable checker on the test row. The check is done in the row checker below.
+                .withChecker((testRows, actualRows) -> { })
+                .withRowChecker(sparkRows -> {
+                    assertEquals("Unexpected number of rows in output", numRows, sparkRows.size());
+                    for (int index = 0, pkValidationIdx = 0; index < sparkRows.size(); index++)
+                    {
+                        Row row = sparkRows.get(index);
+                        long lmtInMillis = row.getTimestamp(numOfColumns).getTime();
+                        assertTrue("Last modification time should have a lower bound of " + minTimestamp,
+                                   lmtInMillis >= minTimestamp);
+                        byte[] updatedFieldsIndicator = (byte[]) row.get(numOfColumns + 1);  // Indicator column
+                        BitSet bs = BitSet.valueOf(updatedFieldsIndicator);
+                        BitSet expected = new BitSet(numOfColumns);
+                        if (partitionDeletionIndices.contains(index))
+                        {
+                            // Verify partition deletion
+                            List<Object> pk = new ArrayList<>(partitionKeys);
+                            // Expecting partition keys
+                            for (int key = 0; key < partitionKeys; key++)
+                            {
+                                expected.set(key);
+                                assertNotNull("partition key should not be null", row.get(key));
+                                pk.add(row.get(key));
+                            }
+                            assertEquals("row" + index + " should only have only the partition keys to be flagged.",
+                                         expected, bs);
+                            List<Object> expectedPK = validationPk.get(pkValidationIdx++);
+                            assertTrue("Partition deletion should indicate the correct partition at row" + index + ". "
+                                     + "Expected: " + expectedPK + ", actual: " + pk,
+                                       ComparisonUtils.equals(expectedPK.toArray(), pk.toArray()));
+                            if (hasClustering)
+                            {
+                                assertNull("ck should be null at row" + index, row.get(partitionKeys));  // ck should be set
+                            }
+                            for (int colIndex = partitionKeys; colIndex < numOfColumns; colIndex++)
+                            {
+                                // null due to partition deletion
+                                assertNull("None partition key columns should be null at row" + 1, row.get(colIndex));
+                            }
+                        }
+                        else
+                        {
+                            // Verify update
+                            for (int colIndex = 0; colIndex < numOfColumns; colIndex++)
+                            {
+                                expected.set(colIndex);
+                                assertNotNull("All column values should exist for full row update",
+                                              row.get(colIndex));
+                            }
+                            assertEquals("row" + index + " should have all columns set", expected, bs);
+                        }
+                    }
+                })
+                .run());
+    }
+
+    @Test
+    public void testElementDeletionInMap()
+    {
+        String name = "m";
+        testElementDeletionInCollection(2,  // Number of columns
+                                        ImmutableList.of(name),
+                                        type -> TestSchema.builder()
+                                                          .withPartitionKey("pk", bridge.uuid())
+                                                          .withColumn(name, bridge.map(type, type)));
+    }
+
+    @Test
+    public void testElementDeletionInSet()
+    {
+        String name = "s";
+        testElementDeletionInCollection(2,  // Number of columns
+                                        ImmutableList.of(name),
+                                        type -> TestSchema.builder()
+                                                          .withPartitionKey("pk", bridge.uuid())
+                                                          .withColumn(name, bridge.set(type)));
+    }
+
+    @Test
+    public void testElementDeletionsInMultipleColumns()
+    {
+        testElementDeletionInCollection(4,  // Number of columns
+                                        ImmutableList.of("c1", "c2", "c3"),
+                                        type -> TestSchema.builder()
+                                                          .withPartitionKey("pk", bridge.uuid())
+                                                          .withColumn("c1", bridge.set(type))
+                                                          .withColumn("c2", bridge.set(type))
+                                                          .withColumn("c3", bridge.set(type)));
+    }
+
+    // Validate that cell deletions in a complex data can be correctly encoded
+    private void testElementDeletionInCollection(int numOfColumns,
+                                                 List<String> collectionColumnNames,
+                                                 Function<CqlField.NativeType, TestSchema.Builder> schemaBuilder)
+    {
+        // Key: row# that has deletion; value: the deleted cell key/path in the collection
+        Map<Integer, byte[]> elementDeletionIndices = new HashMap<>();
+        Random random = new Random(1);
+        long minTimestamp = System.currentTimeMillis();
+        int numRows = 1000;
+        qt().forAll(TestUtils.cql3Type(bridge))
+            .checkAssert(type -> CdcTester
+                .builder(bridge, DIRECTORY, schemaBuilder.apply(type))
+                .withAddLastModificationTime(true)
+                .clearWriters()
+                .withNumRows(numRows)
+                .withWriter((tester, rows, writer) -> {
+                    elementDeletionIndices.clear();
+                    long timestamp = minTimestamp;
+                    for (int row = 0; row < tester.numRows; row++)
+                    {
+                        int ignoredSize = 10;
+                        TestSchema.TestRow testRow;
+                        if (random.nextDouble() < 0.5)
+                        {
+                            // NOTE: This is a little hacky: for simplicity,
+                            //       all collections in the row has the SAME entry being deleted
+                            ByteBuffer key = type.serialize(type.randomValue(ignoredSize));
+                            testRow = Tester.newUniqueRow(tester.schema, rows);
+                            for (String name : collectionColumnNames)
+                            {
+                                testRow = testRow.copy(name, bridge.deletedCollectionElement(key));
+                            }
+                            elementDeletionIndices.put(row, key.array());
+                        }
+                        else
+                        {
+                            testRow = Tester.newUniqueRow(tester.schema, rows);
+                        }
+                        timestamp += 1;
+                        writer.accept(testRow, TimeUnit.MILLISECONDS.toMicros(timestamp));
+                    }
+                })
+                // Disable checker on the test row. The check is done in the sparkrow checker below.
+                .withChecker((testRows, actualRows) -> { })
+                .withRowChecker(sparkRows -> {
+                    assertEquals("Unexpected number of rows in output", numRows, sparkRows.size());
+                    for (int rowIndex = 0; rowIndex < sparkRows.size(); rowIndex++)
+                    {
+                        Row row = sparkRows.get(rowIndex);
+                        long lmtInMillis = row.getTimestamp(numOfColumns).getTime();
+                        assertTrue("Last modification time should have a lower bound of " + minTimestamp,
+                                   lmtInMillis >= minTimestamp);
+                        byte[] updatedFieldsIndicator = (byte[]) row.get(numOfColumns + 1);  // Indicator column
+                        BitSet bs = BitSet.valueOf(updatedFieldsIndicator);
+                        BitSet expected = new BitSet(numOfColumns);
+                        for (int columnIndex = 0; columnIndex < numOfColumns; columnIndex++)
+                        {
+                            expected.set(columnIndex);
+                        }
+                        if (elementDeletionIndices.containsKey(rowIndex))
+                        {
+                            // Verify deletion
+                            Map<Object, Object> cellTombstonesPerColumn = row.getJavaMap(numOfColumns + 3);  // Cell deletion in complex
+                            assertNotNull(cellTombstonesPerColumn);
+                            for (String name : collectionColumnNames)
+                            {
+                                assertNull("Collection column should be null after deletion",
+                                           row.get(row.fieldIndex(name)));
+
+                                assertNotNull(cellTombstonesPerColumn.get(name));
+                                List<?> deletedCellKeys =
+                                        mutableSeqAsJavaList((AbstractSeq<?>) cellTombstonesPerColumn.get(name));
+                                assertEquals(1, deletedCellKeys.size());
+                                byte[] keyBytesRead = (byte[]) deletedCellKeys.get(0);
+                                assertArrayEquals("The key encoded should be the same",
+                                                  elementDeletionIndices.get(rowIndex), keyBytesRead);
+                            }
+                        }
+                        else
+                        {
+                            // Verify update
+                            for (int colIndex = 0; colIndex < numOfColumns; colIndex++)
+                            {
+                                assertNotNull("All column values should exist for full row update",
+                                              row.get(colIndex));
+                            }
+                            assertNull("the cell deletion map should be absent",
+                                       row.get(numOfColumns + 3));
+                        }
+                        assertEquals("row" + rowIndex + " should have all columns set", expected, bs);
+                    }
+                })
+                .run());
+    }
+
+    @Test
+    public void testRangeDeletions()
+    {
+        testRangeDeletions(4,     // Number of columns
+                           1,     // Number of partition key columns
+                           2,     // Number of clustering key columns
+                           true,  // Open end
+                           type -> TestSchema.builder()
+                                             .withPartitionKey("pk1", bridge.uuid())
+                                             .withClusteringKey("ck1", type)
+                                             .withClusteringKey("ck2", bridge.bigint())
+                                             .withColumn("c1", type));
+        testRangeDeletions(4,      // Number of columns
+                           1,      // Number of partition key columns
+                           2,      // Number of clustering key columns
+                           false,  // Open end
+                           type -> TestSchema.builder()
+                                             .withPartitionKey("pk1", bridge.uuid())
+                                             .withClusteringKey("ck1", type)
+                                             .withClusteringKey("ck2", bridge.bigint())
+                                             .withColumn("c1", type));
+    }
+
+    @Test
+    public void testRangeDeletionsWithStatic()
+    {
+        testRangeDeletions(5,     // Number of columns
+                           1,     // Number of partition key columns
+                           2,     // Number of clustering key columns
+                           true,  // Open end
+                           type -> TestSchema.builder()
+                                             .withPartitionKey("pk1", bridge.uuid())
+                                             .withClusteringKey("ck1", bridge.ascii())
+                                             .withClusteringKey("ck2", bridge.bigint())
+                                             .withStaticColumn("s1", bridge.uuid())
+                                             .withColumn("c1", type));
+        testRangeDeletions(5,      // Number of columns
+                           1,      // Number of partition key columns
+                           2,      // Number of clustering key columns
+                           false,  // Open end
+                           type -> TestSchema.builder()
+                                             .withPartitionKey("pk1", bridge.uuid())
+                                             .withClusteringKey("ck1", bridge.ascii())
+                                             .withClusteringKey("ck2", bridge.bigint())
+                                             .withStaticColumn("s1", bridge.uuid())
+                                             .withColumn("c1", type));
+    }
+
+    // Validate that range deletions can be correctly encoded
+    // CHECKSTYLE IGNORE: Long method
+    @SuppressWarnings({"unchecked", "SameParameterValue"})
+    private void testRangeDeletions(int numOfColumns,
+                                    int numOfPartitionKeys,
+                                    int numOfClusteringKeys,
+                                    boolean withOpenEnd,
+                                    Function<CqlField.NativeType, TestSchema.Builder> schemaBuilder)
+    {
+        Preconditions.checkArgument(0 < numOfClusteringKeys,
+                                    "Range deletion test won't run without having clustering keys!");
+        // Key: row# that has deletion; value: the deleted cell key/path in the collection
+        Map<Integer, TestSchema.TestRow> rangeTombestones = new HashMap<>();
+        Random random = new Random(1);
+        long minTimestamp = System.currentTimeMillis();
+        int numRows = 1000;
+        qt().forAll(TestUtils.cql3Type(bridge))
+            .checkAssert(type -> CdcTester
+                .builder(bridge, DIRECTORY, schemaBuilder.apply(type))
+                .withAddLastModificationTime(true)
+                .clearWriters()
+                .withNumRows(numRows)
+                .withWriter((tester, rows, writer) -> {
+                    long timestamp = minTimestamp;
+                    rangeTombestones.clear();
+                    for (int row = 0; row < tester.numRows; row++)
+                    {
+                        TestSchema.TestRow testRow;
+                        if (random.nextDouble() < 0.5)
+                        {
+                            testRow = Tester.newUniqueRow(tester.schema, rows);
+                            Object[] baseBound =
+                                    testRow.rawValues(numOfPartitionKeys, numOfPartitionKeys + numOfClusteringKeys);
+                            // Create a new bound that has the last ck value different from the base bound
+                            Object[] newBound = new Object[baseBound.length];
+                            System.arraycopy(baseBound, 0, newBound, 0, baseBound.length);
+                            TestSchema.TestRow newRow = Tester.newUniqueRow(tester.schema, rows);
+                            int lastCK = newBound.length - 1;
+                            newBound[lastCK] = newRow.get(numOfPartitionKeys + numOfClusteringKeys - 1);
+                            Object[] open;
+                            Object[] close;
+                            // The field's corresponding java type should be comparable...
+                            if (((Comparable<Object>) baseBound[lastCK]).compareTo(newBound[lastCK]) < 0)  // For queries like WHERE ck > 1 AND ck < 2
+                            {
+                                open = baseBound;
+                                close = newBound;
+                            }
+                            else
+                            {
+                                open = newBound;
+                                close = baseBound;
+                            }
+                            if (withOpenEnd)  // For queries like WHERE ck > 1
+                            {
+                                close[lastCK] = null;
+                            }
+                            testRow.setRangeTombstones(ImmutableList.of(new RangeTombstone(
+                                    new RangeTombstone.Bound(open, true), new RangeTombstone.Bound(close, true))));
+                            rangeTombestones.put(row, testRow);
+                        }
+                        else
+                        {
+                            testRow = Tester.newUniqueRow(tester.schema, rows);
+                        }
+                        timestamp += 1;
+                        writer.accept(testRow, TimeUnit.MILLISECONDS.toMicros(timestamp));
+                    }
+                })
+                // Disable checker on the test row. The check is done in the sparkrow checker below.
+                .withChecker((testRows, actualRows) -> { })
+                .withSparkRowTestRowsChecker((testRows, sparkRows) -> {
+                    assertEquals("Unexpected number of rows in output", numRows, sparkRows.size());
+                    for (int rowIndex = 0; rowIndex < sparkRows.size(); rowIndex++)
+                    {
+                        Row row = sparkRows.get(rowIndex);
+                        long lmtInMillis = row.getTimestamp(numOfColumns).getTime();
+                        assertTrue("Last modification time should have a lower bound of " + minTimestamp,
+                                   minTimestamp <= lmtInMillis);
+                        byte[] updatedFieldsIndicator = (byte[]) row.get(numOfColumns + 1);  // Indicator column
+                        BitSet bs = BitSet.valueOf(updatedFieldsIndicator);
+                        BitSet expected = new BitSet(numOfColumns);
+                        if (rangeTombestones.containsKey(rowIndex))  // Verify deletion
+                        {
+                            for (int column = 0; column < numOfColumns; column++)
+                            {
+                                if (column < numOfPartitionKeys)
+                                {
+                                    assertNotNull("All partition keys should exist for range tombstone",
+                                                  row.get(column));
+                                    expected.set(column);
+                                }
+                                else
+                                {
+                                    assertNull("Non-partition key columns should be null",
+                                               row.get(column));
+                                }
+                                Object deletionColumn = row.get(numOfColumns + 4);  // Range deletion column
+                                assertNotNull(deletionColumn);
+                                List<?> tombstones = mutableSeqAsJavaList((AbstractSeq<?>) deletionColumn);
+                                assertEquals("There should be 1 range tombstone", 1, tombstones.size());
+                                TestSchema.TestRow sourceRow = rangeTombestones.get(rowIndex);
+                                RangeTombstone expectedTombstone = sourceRow.rangeTombstones().get(0);
+                                Row rangeTombstone = (Row) tombstones.get(0);
+                                assertEquals("Range tombstone should have 4 fields", 4, rangeTombstone.length());
+                                assertEquals(expectedTombstone.open.inclusive, rangeTombstone.getAs("StartInclusive"));
+                                assertEquals(expectedTombstone.close.inclusive, rangeTombstone.getAs("EndInclusive"));
+                                Row open = rangeTombstone.getAs("Start");
+                                assertEquals(numOfClusteringKeys, open.length());
+                                Row close = rangeTombstone.getAs("End");
+                                assertEquals(numOfClusteringKeys, close.length());
+                                for (int keyIndex = 0; keyIndex < numOfClusteringKeys; keyIndex++)
+                                {
+                                    assertEqualsWithMessage(expectedTombstone.open.values[keyIndex],
+                                                            open.get(keyIndex));
+                                    assertEqualsWithMessage(expectedTombstone.close.values[keyIndex],
+                                                            close.get(keyIndex));
+                                }
+                            }
+                        }
+                        else  // Verify update
+                        {
+                            for (int columnIndex = 0; columnIndex < numOfColumns; columnIndex++)
+                            {
+                                assertNotNull("All column values should exist for full row update",
+                                              row.get(columnIndex));
+                                expected.set(columnIndex);
+                            }
+                            assertNull("the cell deletion map should be absent",
+                                       row.get(numOfColumns + 3));
+                        }
+                        assertEquals("row" + rowIndex + " should have the expected columns set", expected, bs);
+                    }
+                })
+                .run());
+    }
+}
diff --git a/cassandra-analytics-core/src/test/spark3/org/apache/cassandra/spark/cdc/RequireTwoReplicasLocalDataSource.java b/cassandra-analytics-core/src/test/spark3/org/apache/cassandra/spark/cdc/RequireTwoReplicasLocalDataSource.java
new file mode 100644
index 0000000..5280e43
--- /dev/null
+++ b/cassandra-analytics-core/src/test/spark3/org/apache/cassandra/spark/cdc/RequireTwoReplicasLocalDataSource.java
@@ -0,0 +1,45 @@
+/*
+ * 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.cassandra.spark.cdc;
+
+import java.io.Serializable;
+
+import org.apache.cassandra.spark.data.DataLayer;
+import org.apache.cassandra.spark.data.LocalDataLayer;
+import org.apache.cassandra.spark.sparksql.CassandraTableProvider;
+import org.apache.spark.sql.util.CaseInsensitiveStringMap;
+
+// Only used for testing
+public class RequireTwoReplicasLocalDataSource extends CassandraTableProvider implements Serializable
+{
+    @Override
+    public String shortName()
+    {
+        return "LocalDataSourceRequiresAtLeastTwoReplicasToCompact";
+    }
+
+    @Override
+    public DataLayer getDataLayer(CaseInsensitiveStringMap options)
+    {
+        // Overrides MinimumReplicasPerMutation for testing
+        return LocalDataLayer.from(options)
+                             .withMinimumReplicasPerMutation(2);
+    }
+}
diff --git a/cassandra-analytics-core/src/test/spark3/org/apache/cassandra/spark/sparksql/CdcOffsetTests.java b/cassandra-analytics-core/src/test/spark3/org/apache/cassandra/spark/sparksql/CdcOffsetTests.java
new file mode 100644
index 0000000..0cea310
--- /dev/null
+++ b/cassandra-analytics-core/src/test/spark3/org/apache/cassandra/spark/sparksql/CdcOffsetTests.java
@@ -0,0 +1,42 @@
+/*
+ * 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.cassandra.spark.sparksql;
+
+import java.io.IOException;
+
+import org.junit.Test;
+
+import org.apache.cassandra.spark.sparksql.filters.CdcOffset;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+public class CdcOffsetTests
+{
+    @Test
+    public void testJsonSerialization() throws IOException
+    {
+        CdcOffset offset = new CdcOffset(500L);
+        String json = CdcOffset.MAPPER.writeValueAsString(offset);
+        assertNotNull(json);
+        CdcOffset deserialized = CdcOffset.fromJson(json);
+        assertEquals(offset, deserialized);
+    }
+}
diff --git a/cassandra-bridge/build.gradle b/cassandra-bridge/build.gradle
new file mode 100644
index 0000000..838944b
--- /dev/null
+++ b/cassandra-bridge/build.gradle
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+project(':cassandra-bridge') {
+    apply(plugin: 'java-library')
+
+    dependencies {
+        compileOnly(group: "${sparkGroupId}", name: "spark-core_${scalaMajorVersion}", version: "${project.rootProject.sparkVersion}")
+        compileOnly(group: "${sparkGroupId}", name: "spark-sql_${scalaMajorVersion}", version: "${project.rootProject.sparkVersion}")
+
+        testImplementation(group: 'com.fasterxml.jackson.core', name: 'jackson-databind', version: "${jacksonVersion}")
+        testImplementation(group: 'com.google.guava', name: 'guava', version: '31.1-jre')
+        testImplementation(group: 'org.slf4j', name: 'slf4j-simple', version: '1.7.26')
+        testImplementation(group: 'junit', name: 'junit', version: "${project.rootProject.junitVersion}")
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/bridge/BigNumberConfig.java b/cassandra-bridge/src/main/java/org/apache/cassandra/bridge/BigNumberConfig.java
new file mode 100644
index 0000000..af5767e
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/bridge/BigNumberConfig.java
@@ -0,0 +1,54 @@
+/*
+ * 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.cassandra.bridge;
+
+public interface BigNumberConfig
+{
+    BigNumberConfig DEFAULT = new BigNumberConfig()
+    {
+        public int bigIntegerPrecision()
+        {
+            return 38;
+        }
+
+        public int bigIntegerScale()
+        {
+            return 0;
+        }
+
+        public int bigDecimalPrecision()
+        {
+            return 38;
+        }
+
+        public int bigDecimalScale()
+        {
+            return 19;
+        }
+    };
+
+    int bigIntegerPrecision();
+
+    int bigIntegerScale();
+
+    int bigDecimalPrecision();
+
+    int bigDecimalScale();
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/bridge/BigNumberConfigImpl.java b/cassandra-bridge/src/main/java/org/apache/cassandra/bridge/BigNumberConfigImpl.java
new file mode 100644
index 0000000..3ed911a
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/bridge/BigNumberConfigImpl.java
@@ -0,0 +1,187 @@
+/*
+ * 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.cassandra.bridge;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.cassandra.spark.utils.MapUtils;
+
+/**
+ * An implementation of the {@link BigNumberConfig} that supports JDK, Kryo, as well as jackson serialization
+ */
+public class BigNumberConfigImpl implements BigNumberConfig, Serializable
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(BigNumberConfigImpl.class);
+
+    private int bigIntegerPrecision;
+    private int bigIntegerScale;
+    private int bigDecimalPrecision;
+    private int bigDecimalScale;
+
+    @JsonCreator
+    public BigNumberConfigImpl(@JsonProperty("bigIntegerPrecision") int bigIntegerPrecision,
+                               @JsonProperty("bigIntegerScale")     int bigIntegerScale,
+                               @JsonProperty("bigDecimalPrecision") int bigDecimalPrecision,
+                               @JsonProperty("bigDecimalScale")     int bigDecimalScale)
+    {
+        this.bigIntegerPrecision = bigIntegerPrecision;
+        this.bigIntegerScale     = bigIntegerScale;
+        this.bigDecimalPrecision = bigDecimalPrecision;
+        this.bigDecimalScale     = bigDecimalScale;
+    }
+
+    public static BigNumberConfigImpl of(int bigIntegerPrecision,
+                                         int bigIntegerScale,
+                                         int bigDecimalPrecision,
+                                         int bigDecimalScale)
+    {
+        return new BigNumberConfigImpl(bigIntegerPrecision, bigIntegerScale, bigDecimalPrecision, bigDecimalScale);
+    }
+
+    public static Map<String, BigNumberConfigImpl> build(Map<String, String> options)
+    {
+        return Optional.ofNullable(options.get(MapUtils.lowerCaseKey("bigNumberConfig")))
+                       .map(BigNumberConfigImpl::build)
+                       .orElse(Collections.emptyMap());
+    }
+
+    public static Map<String, BigNumberConfigImpl> build(String bigNumberConfig)
+    {
+        TypeReference<HashMap<String, BigNumberConfigImpl>> typeRef =
+            new TypeReference<HashMap<String, BigNumberConfigImpl>>() {};  // CHECKSTYLE IGNORE: Empty anonymous inner class
+        try
+        {
+            return new ObjectMapper().readValue(bigNumberConfig, typeRef);
+        }
+        catch (IOException exception)
+        {
+            LOGGER.error("IOException could not read BigNumberConfig json", exception);
+            throw new RuntimeException(exception);
+        }
+    }
+
+    @JsonProperty
+    public int bigIntegerPrecision()
+    {
+        return bigIntegerPrecision;
+    }
+
+    @JsonProperty
+    public int bigIntegerScale()
+    {
+        return bigIntegerScale;
+    }
+
+    @JsonProperty
+    public int bigDecimalPrecision()
+    {
+        return bigDecimalPrecision;
+    }
+
+    @JsonProperty
+    public int bigDecimalScale()
+    {
+        return bigDecimalScale;
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Objects.hash(super.hashCode(),
+                            bigIntegerPrecision,
+                            bigIntegerScale,
+                            bigDecimalPrecision,
+                            bigDecimalScale);
+    }
+
+    @Override
+    public boolean equals(Object other)
+    {
+        if (this == other)
+        {
+            return true;
+        }
+        if (other == null || this.getClass() != other.getClass())
+        {
+            return false;
+        }
+
+        BigNumberConfigImpl that = (BigNumberConfigImpl) other;
+        return bigIntegerPrecision == that.bigIntegerPrecision
+            && bigIntegerScale == that.bigIntegerScale
+            && bigDecimalPrecision == that.bigDecimalPrecision
+            && bigDecimalScale == that.bigDecimalScale;
+    }
+
+    // JDK Serialization
+
+    private void readObject(ObjectInputStream in) throws IOException
+    {
+        bigIntegerPrecision = in.readInt();
+        bigIntegerScale = in.readInt();
+        bigDecimalPrecision = in.readInt();
+        bigDecimalScale = in.readInt();
+    }
+
+    private void writeObject(ObjectOutputStream out) throws IOException
+    {
+        out.writeInt(bigIntegerPrecision);
+        out.writeInt(bigIntegerScale);
+        out.writeInt(bigDecimalPrecision);
+        out.writeInt(bigDecimalScale);
+    }
+
+    // Kryo Serialization
+
+    public static class Serializer extends com.esotericsoftware.kryo.Serializer<BigNumberConfigImpl>
+    {
+
+        public void write(Kryo kryo, Output out, BigNumberConfigImpl bigNumberConfig)
+        {
+            out.writeInt(bigNumberConfig.bigIntegerPrecision);
+            out.writeInt(bigNumberConfig.bigIntegerScale);
+            out.writeInt(bigNumberConfig.bigDecimalPrecision);
+            out.writeInt(bigNumberConfig.bigDecimalScale);
+        }
+
+        public BigNumberConfigImpl read(Kryo kryo, Input input, Class<BigNumberConfigImpl> type)
+        {
+            return BigNumberConfigImpl.of(input.readInt(), input.readInt(), input.readInt(), input.readInt());
+        }
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/bridge/CassandraBridge.java b/cassandra-bridge/src/main/java/org/apache/cassandra/bridge/CassandraBridge.java
new file mode 100644
index 0000000..81e11b7
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/bridge/CassandraBridge.java
@@ -0,0 +1,561 @@
+/*
+ * 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.cassandra.bridge;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.nio.file.Path;
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.function.Consumer;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableMap;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.Serializer;
+import com.esotericsoftware.kryo.io.Input;
+import org.apache.cassandra.spark.cdc.CommitLog;
+import org.apache.cassandra.spark.cdc.CommitLogProvider;
+import org.apache.cassandra.spark.cdc.IPartitionUpdateWrapper;
+import org.apache.cassandra.spark.cdc.TableIdLookup;
+import org.apache.cassandra.spark.cdc.watermarker.Watermarker;
+import org.apache.cassandra.spark.data.CqlField;
+import org.apache.cassandra.spark.data.CqlTable;
+import org.apache.cassandra.spark.data.ReplicationFactor;
+import org.apache.cassandra.spark.data.SSTablesSupplier;
+import org.apache.cassandra.spark.data.partitioner.Partitioner;
+import org.apache.cassandra.spark.reader.StreamScanner;
+import org.apache.cassandra.spark.sparksql.filters.CdcOffsetFilter;
+import org.apache.cassandra.spark.sparksql.filters.PartitionKeyFilter;
+import org.apache.cassandra.spark.sparksql.filters.PruneColumnFilter;
+import org.apache.cassandra.spark.sparksql.filters.SparkRangeFilter;
+import org.apache.cassandra.spark.stats.Stats;
+import org.apache.cassandra.spark.utils.TimeProvider;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Provides an abstract interface for all calls to the Cassandra code of a specific version
+ */
+@SuppressWarnings({ "WeakerAccess", "unused" })
+public abstract class CassandraBridge
+{
+    // Used to indicate if a column is unset; used in generating mutations for CommitLog
+    @VisibleForTesting
+    public static final Object UNSET_MARKER = new Object();
+
+    public static final Pattern COLLECTION_PATTERN = Pattern.compile("^(set|list|map|tuple)<(.+)>$", Pattern.CASE_INSENSITIVE);
+    public static final Pattern FROZEN_PATTERN = Pattern.compile("^frozen<(.*)>$", Pattern.CASE_INSENSITIVE);
+
+    public abstract AbstractMap.SimpleEntry<ByteBuffer, BigInteger> getPartitionKey(@NotNull CqlTable table,
+                                                                                    @NotNull Partitioner partitioner,
+                                                                                    @NotNull List<String> keys);
+
+    public abstract TimeProvider timeProvider();
+
+    // CDC Stream Scanner
+    // CHECKSTYLE IGNORE: Method with many parameters
+    public abstract StreamScanner getCdcScanner(int partitionId,
+                                                @NotNull CqlTable table,
+                                                @NotNull Partitioner partitioner,
+                                                @NotNull CommitLogProvider commitLogProvider,
+                                                @NotNull TableIdLookup tableIdLookup,
+                                                @NotNull Stats stats,
+                                                @Nullable SparkRangeFilter sparkRangeFilter,
+                                                @Nullable CdcOffsetFilter offset,
+                                                int minimumReplicasPerMutation,
+                                                @NotNull Watermarker watermarker,
+                                                @NotNull String jobId,
+                                                @NotNull ExecutorService executorService,
+                                                @NotNull TimeProvider timeProvider);
+
+    // Compaction Stream Scanner
+    // CHECKSTYLE IGNORE: Method with many parameters
+    public abstract StreamScanner getCompactionScanner(@NotNull CqlTable table,
+                                                       @NotNull Partitioner partitionerType,
+                                                       @NotNull SSTablesSupplier ssTables,
+                                                       @Nullable SparkRangeFilter sparkRangeFilter,
+                                                       @NotNull Collection<PartitionKeyFilter> partitionKeyFilters,
+                                                       @Nullable PruneColumnFilter columnFilter,
+                                                       @NotNull TimeProvider timeProvider,
+                                                       boolean readIndexOffset,
+                                                       boolean useIncrementalRepair,
+                                                       @NotNull Stats stats);
+
+    public abstract CassandraVersion getVersion();
+
+    public abstract BigInteger hash(Partitioner partitioner, ByteBuffer key);
+
+    public abstract UUID getTimeUUID();
+
+    // CQL Schema
+
+    @VisibleForTesting
+    public CqlTable buildSchema(String createStatement, String keyspace)
+    {
+        return buildSchema(createStatement,
+                           keyspace,
+                           new ReplicationFactor(ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy,
+                                                 ImmutableMap.of("DC1", 3)));
+    }
+
+    @VisibleForTesting
+    public CqlTable buildSchema(String createStatement, String keyspace, ReplicationFactor replicationFactor)
+    {
+        return buildSchema(createStatement, keyspace, replicationFactor, Partitioner.Murmur3Partitioner);
+    }
+
+    @VisibleForTesting
+    public CqlTable buildSchema(String createStatement,
+                                String keyspace,
+                                ReplicationFactor replicationFactor,
+                                Partitioner partitioner)
+    {
+        return buildSchema(createStatement, keyspace, replicationFactor, partitioner, Collections.emptySet());
+    }
+
+    @VisibleForTesting
+    public CqlTable buildSchema(String createStatement,
+                                String keyspace,
+                                ReplicationFactor replicationFactor,
+                                Partitioner partitioner,
+                                Set<String> udts)
+    {
+        return buildSchema(createStatement, keyspace, replicationFactor, partitioner, udts, null, 0);
+    }
+
+    public abstract CqlTable buildSchema(String createStatement,
+                                         String keyspace,
+                                         ReplicationFactor replicationFactor,
+                                         Partitioner partitioner,
+                                         Set<String> udts,
+                                         @Nullable UUID tableId,
+                                         int indexCount);
+
+    // CQL Type Parsing
+
+    public abstract CqlField.CqlType readType(CqlField.CqlType.InternalType type, Input input);
+
+    public List<CqlField.NativeType> allTypes()
+    {
+        return Arrays.asList(ascii(), bigint(), blob(), bool(), counter(), date(), decimal(), aDouble(),
+                             duration(), empty(), aFloat(), inet(), aInt(), smallint(), text(), time(),
+                             timestamp(), timeuuid(), tinyint(), uuid(), varchar(), varint());
+    }
+
+    public abstract Map<String, ? extends CqlField.NativeType> nativeTypeNames();
+
+    public CqlField.NativeType nativeType(String name)
+    {
+        return nativeTypeNames().get(name.toLowerCase());
+    }
+
+    public List<CqlField.NativeType> supportedTypes()
+    {
+        return allTypes().stream().filter(CqlField.NativeType::isSupported).collect(Collectors.toList());
+    }
+
+    // Native
+
+    public abstract CqlField.NativeType ascii();
+
+    public abstract CqlField.NativeType blob();
+
+    public abstract CqlField.NativeType bool();
+
+    public abstract CqlField.NativeType counter();
+
+    public abstract CqlField.NativeType bigint();
+
+    public abstract CqlField.NativeType date();
+
+    public abstract CqlField.NativeType decimal();
+
+    public abstract CqlField.NativeType aDouble();
+
+    public abstract CqlField.NativeType duration();
+
+    public abstract CqlField.NativeType empty();
+
+    public abstract CqlField.NativeType aFloat();
+
+    public abstract CqlField.NativeType inet();
+
+    public abstract CqlField.NativeType aInt();
+
+    public abstract CqlField.NativeType smallint();
+
+    public abstract CqlField.NativeType text();
+
+    public abstract CqlField.NativeType time();
+
+    public abstract CqlField.NativeType timestamp();
+
+    public abstract CqlField.NativeType timeuuid();
+
+    public abstract CqlField.NativeType tinyint();
+
+    public abstract CqlField.NativeType uuid();
+
+    public abstract CqlField.NativeType varchar();
+
+    public abstract CqlField.NativeType varint();
+
+    // Complex
+
+    public abstract CqlField.CqlType collection(String name, CqlField.CqlType... types);
+
+    public abstract CqlField.CqlList list(CqlField.CqlType type);
+
+    public abstract CqlField.CqlSet set(CqlField.CqlType type);
+
+    public abstract CqlField.CqlMap map(CqlField.CqlType keyType, CqlField.CqlType valueType);
+
+    public abstract CqlField.CqlTuple tuple(CqlField.CqlType... types);
+
+    public abstract CqlField.CqlType frozen(CqlField.CqlType type);
+
+    public abstract CqlField.CqlUdtBuilder udt(String keyspace, String name);
+
+    public CqlField.CqlType parseType(String type)
+    {
+        return parseType(type, Collections.emptyMap());
+    }
+
+    public CqlField.CqlType parseType(String type, Map<String, CqlField.CqlUdt> udts)
+    {
+        if (type == null || type.length() == 0)
+        {
+            return null;
+        }
+        Matcher collectionMatcher = COLLECTION_PATTERN.matcher(type);
+        if (collectionMatcher.find())
+        {
+            // CQL collection
+            String[] types = splitInnerTypes(collectionMatcher.group(2));
+            return collection(collectionMatcher.group(1), Stream.of(types)
+                                                                .map(collectionType -> parseType(collectionType, udts))
+                                                                .toArray(CqlField.CqlType[]::new));
+        }
+        Matcher frozenMatcher = FROZEN_PATTERN.matcher(type);
+        if (frozenMatcher.find())
+        {
+            // Frozen collections
+            return frozen(parseType(frozenMatcher.group(1), udts));
+        }
+
+        if (udts.containsKey(type))
+        {
+            // User-defined type
+            return udts.get(type);
+        }
+
+        // Native CQL 3 type
+        return nativeType(type);
+    }
+
+    @VisibleForTesting
+    public static String[] splitInnerTypes(String str)
+    {
+        List<String> result = new ArrayList<>();
+        StringBuilder current = new StringBuilder();
+        int parentheses = 0;
+        for (int index = 0; index < str.length(); index++)
+        {
+            char character = str.charAt(index);
+            switch (character)
+            {
+                case ' ':
+                    if (parentheses == 0)
+                    {
+                        continue;
+                    }
+                    break;
+                case ',':
+                    if (parentheses == 0)
+                    {
+                        if (current.length() > 0)
+                        {
+                            result.add(current.toString());
+                            current = new StringBuilder();
+                        }
+                        continue;
+                    }
+                    break;
+                case '<':
+                    parentheses++;
+                    break;
+                case '>':
+                    parentheses--;
+                    break;
+                default:
+                    // Do nothing
+            }
+            current.append(character);
+        }
+
+        if (current.length() > 0 || result.isEmpty())
+        {
+            result.add(current.toString());
+        }
+
+        return result.toArray(new String[0]);
+    }
+
+    // SSTable Writer
+
+    @FunctionalInterface
+    public interface Writer
+    {
+        void write(Object... values);
+    }
+
+    public void writeSSTable(Partitioner partitioner,
+                             String keyspace,
+                             String table,
+                             Path directory,
+                             String createStatement,
+                             String insertStatement,
+                             Consumer<Writer> writer)
+    {
+        writeSSTable(partitioner,
+                     keyspace,
+                     table,
+                     directory,
+                     createStatement,
+                     insertStatement,
+                     null,
+                     false,
+                     Collections.emptySet(),
+                     writer);
+    }
+
+    // CHECKSTYLE IGNORE: Method with many parameters
+    public abstract void writeSSTable(Partitioner partitioner,
+                                      String keyspace,
+                                      String table,
+                                      Path directory,
+                                      String createStatement,
+                                      String insertStatement,
+                                      String updateStatement,
+                                      boolean upsert,
+                                      Set<CqlField.CqlUdt> udts,
+                                      Consumer<Writer> writer);
+
+    public abstract SSTableWriter getSSTableWriter(String inDirectory,
+                                                   String partitioner,
+                                                   String createStatement,
+                                                   String insertStatement,
+                                                   boolean isSorted,
+                                                   int bufferSizeMB);
+
+    // CDC Configuration
+
+    public abstract void setCDC(Path path);
+
+    public abstract void setCommitLogPath(Path path);
+
+    @VisibleForTesting
+    public abstract ICommitLog testCommitLog(File folder);
+
+    // CommitLog
+
+    public interface IMutation
+    {
+    }
+
+    public interface IRow
+    {
+        Object get(int position);
+
+        /**
+         * @return true if the entire row is deleted, false otherwise
+         */
+        default boolean isDeleted()
+        {
+            return false;
+        }
+
+        /**
+         * @return true if the row is from an INSERT statement, false otherwise
+         */
+        default boolean isInsert()
+        {
+            return true;
+        }
+
+        /**
+         * Get the range tombstones for this partition
+         *
+         * TODO: IRow is used as a partition; semantically, it does not fit
+         *
+         * @return null if no range tombstones exist. Otherwise, return a list of range tombstones
+         */
+        default List<RangeTombstone> rangeTombstones()
+        {
+            return null;
+        }
+    }
+
+    public interface ICommitLog
+    {
+        void start();
+
+        void stop();
+
+        void clear();
+
+        void add(IMutation mutation);
+
+        void sync();
+    }
+
+    /**
+     * Cassandra-version-specific implementation for logging a row mutation to CommitLog.
+     * Used for CDC unit test framework.`
+     *
+     * @param table     CQL table schema
+     * @param log       CommitLog instance
+     * @param row       row instance
+     * @param timestamp mutation timestamp
+     */
+    @VisibleForTesting
+    public abstract void log(CqlTable table, ICommitLog log, IRow row, long timestamp);
+
+    /**
+     * Determine whether a row is a partition deletion.
+     * It is a partition deletion, when all fields except the partition keys are null.
+     *
+     * @param table CQL table schema
+     * @param row   row instance
+     * @return true if it is a partition deletion
+     */
+    protected abstract boolean isPartitionDeletion(CqlTable table, IRow row);
+
+    /**
+     * Determine whether a row is a row deletion.
+     * It is a row deletion, when all fields except the parimary keys are null.
+     *
+     * @param table CQL table schema
+     * @param row   row instance
+     * @return true if it is a row deletion
+     */
+    protected abstract boolean isRowDeletion(CqlTable table, IRow row);
+
+    @VisibleForTesting
+    public abstract Object livingCollectionElement(ByteBuffer cellPath, Object value);
+
+    @VisibleForTesting
+    public abstract Object deletedCollectionElement(ByteBuffer cellPath);
+
+    @VisibleForTesting
+    public abstract Set<Long> readLog(CqlTable table, CommitLog log, Watermarker watermarker);
+
+    // Version-Specific Test Utility Methods
+
+    @VisibleForTesting
+    public abstract void writeTombstoneSSTable(Partitioner partitioner,
+                                               Path directory,
+                                               String createStatement,
+                                               String deleteStatement,
+                                               Consumer<Writer> writer);
+
+    @VisibleForTesting
+    public abstract void sstableToJson(Path dataDbFile, OutputStream output) throws FileNotFoundException;
+
+    @VisibleForTesting
+    public abstract Object toTupleValue(CqlField.CqlTuple type, Object[] values);
+
+    @VisibleForTesting
+    public abstract Object toUserTypeValue(CqlField.CqlUdt type, Map<String, Object> values);
+
+    // Compression Utils
+
+    public abstract ByteBuffer compress(byte[] bytes) throws IOException;
+
+    public abstract ByteBuffer compress(ByteBuffer input) throws IOException;
+
+    public abstract ByteBuffer uncompress(byte[] bytes) throws IOException;
+
+    public abstract ByteBuffer uncompress(ByteBuffer input) throws IOException;
+
+    // Kryo Serializers
+
+    public abstract Serializer<? extends IPartitionUpdateWrapper> getPartitionUpdateSerializer(
+            String keyspace,
+            String table,
+            boolean includePartitionUpdate);
+
+    // Kryo/Java (De-)Serialization
+
+    public abstract void kryoRegister(Kryo kryo);
+
+    public abstract void javaSerialize(ObjectOutputStream out, Serializable object);
+
+    public abstract <T> T javaDeserialize(ObjectInputStream in, Class<T> type);
+
+    public byte[] javaSerialize(Serializable object)
+    {
+        try (ByteArrayOutputStream bytes = new ByteArrayOutputStream(512);
+             ObjectOutputStream out = new ObjectOutputStream(bytes))
+        {
+            javaSerialize(out, object);
+            return bytes.toByteArray();
+        }
+        catch (IOException exception)
+        {
+            throw new RuntimeException(exception);
+        }
+    }
+
+    public <T> T javaDeserialize(byte[] bytes, Class<T> type)
+    {
+        try (ObjectInputStream in = new ObjectInputStream(new ByteArrayInputStream(bytes)))
+        {
+            return javaDeserialize(in, type);
+        }
+        catch (IOException exception)
+        {
+            throw new RuntimeException(exception);
+        }
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/bridge/CassandraVersion.java b/cassandra-bridge/src/main/java/org/apache/cassandra/bridge/CassandraVersion.java
new file mode 100644
index 0000000..e8bb5c3
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/bridge/CassandraVersion.java
@@ -0,0 +1,69 @@
+/*
+ * 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.cassandra.bridge;
+
+/*
+ * An enum that describes all possible Cassandra versions that can potentially be supported, even if the bridge is not yet implemented.
+ * Customers of this library looking to implement additional bridges or replace existing ones with proprietary implementations
+ * should inject/replace bridge implementation JArs embedded into this library's resources and replace this class with an identical one,
+ * but with implementedVersions() and supportedVersions() modified accordingly.
+ */
+public enum CassandraVersion
+{
+    THREEZERO(30, "3.0", "three-zero"),
+    FOURZERO(40, "4.0", "four-zero"),
+    FOURONE(41, "4.1", "four-zero");
+
+    private final int number;
+    private final String name;
+    private final String jarBaseName;  // Must match shadowJar.archiveFileName from Gradle configuration (without extension)
+
+    CassandraVersion(int number, String name, String jarBaseName)
+    {
+        this.number = number;
+        this.name = name;
+        this.jarBaseName = jarBaseName;
+    }
+
+    public int versionNumber()
+    {
+        return number;
+    }
+
+    public String versionName()
+    {
+        return name;
+    }
+
+    public String jarBaseName()
+    {
+        return jarBaseName;
+    }
+
+    public static CassandraVersion[] implementedVersions()
+    {
+        return new CassandraVersion[]{FOURZERO};
+    }
+
+    public static String[] supportedVersions()
+    {
+        return new String[]{"cassandra-4.0.2"};
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/bridge/RangeTombstone.java b/cassandra-bridge/src/main/java/org/apache/cassandra/bridge/RangeTombstone.java
new file mode 100644
index 0000000..2b7eb84
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/bridge/RangeTombstone.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.cassandra.bridge;
+
+public class RangeTombstone
+{
+    public final Bound open;
+    public final Bound close;
+
+    public RangeTombstone(Bound open, Bound close)
+    {
+        this.open = open;
+        this.close = close;
+    }
+
+    public static class Bound
+    {
+        public final Object[] values;
+        public final boolean inclusive;
+
+        public Bound(Object[] values, boolean inclusive)
+        {
+            this.values = values;
+            this.inclusive = inclusive;
+        }
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/bridge/SSTableWriter.java b/cassandra-bridge/src/main/java/org/apache/cassandra/bridge/SSTableWriter.java
new file mode 100644
index 0000000..9ae1d52
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/bridge/SSTableWriter.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.cassandra.bridge;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+public interface SSTableWriter extends Closeable
+{
+    void addRow(Object... values) throws IOException;
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/bridge/TokenRange.java b/cassandra-bridge/src/main/java/org/apache/cassandra/bridge/TokenRange.java
new file mode 100644
index 0000000..355a920
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/bridge/TokenRange.java
@@ -0,0 +1,154 @@
+/*
+ * 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.cassandra.bridge;
+
+import java.io.Serializable;
+import java.math.BigInteger;
+
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * This is a simple implementation of a range between two {@link BigInteger} tokens.
+ * It allows us avoid dependency on Guava's {@link com.google.common.collect.Range}&lt;{@link BigInteger}&gt;
+ * in the interface of Cassandre Bridge (which had brought us nothing but grief in the past).
+ */
+public final class TokenRange implements Serializable
+{
+    public static final long serialVersionUID = 42L;
+
+    // NOTE: Internally, ranges are always of the closed-open kind
+    @NotNull
+    private final BigInteger lowerBound;
+    @NotNull
+    private final BigInteger upperBound;
+
+    private TokenRange(@NotNull BigInteger lowerBound, @NotNull BigInteger upperBound)
+    {
+        this.lowerBound = lowerBound;
+        this.upperBound = upperBound;
+    }
+
+    @NotNull
+    public BigInteger lowerEndpoint()
+    {
+        return lowerBound;
+    }
+
+    @NotNull
+    public BigInteger upperEndpoint()
+    {
+        return upperBound.subtract(BigInteger.ONE);
+    }
+
+    @NotNull
+    public static TokenRange singleton(@NotNull BigInteger value)
+    {
+        return new TokenRange(value, value.add(BigInteger.ONE));
+    }
+
+    @NotNull
+    public static TokenRange open(@NotNull BigInteger lower, @NotNull BigInteger upper)
+    {
+        return new TokenRange(lower.add(BigInteger.ONE), upper);
+    }
+
+    @NotNull
+    public static TokenRange closed(@NotNull BigInteger lower, @NotNull BigInteger upper)
+    {
+        return new TokenRange(lower, upper.add(BigInteger.ONE));
+    }
+
+    @NotNull
+    public static TokenRange closedOpen(@NotNull BigInteger lower, @NotNull BigInteger upper)
+    {
+        return new TokenRange(lower, upper);
+    }
+
+    @NotNull
+    public static TokenRange openClosed(@NotNull BigInteger lower, @NotNull BigInteger upper)
+    {
+        return new TokenRange(lower.add(BigInteger.ONE), upper.add(BigInteger.ONE));
+    }
+
+    @NotNull
+    public static TokenRange merge(@NotNull TokenRange first, @NotNull TokenRange second)
+    {
+        return first.span(second);
+    }
+
+    public boolean isEmpty()
+    {
+        return lowerBound.compareTo(upperBound) >= 0;
+    }
+
+    public BigInteger size()
+    {
+        return upperBound.max(lowerBound).subtract(lowerBound);
+    }
+
+    public boolean contains(@NotNull BigInteger value)
+    {
+        return lowerBound.compareTo(value) <= 0 && value.compareTo(upperBound) < 0;
+    }
+
+    public boolean encloses(@NotNull TokenRange other)
+    {
+        return lowerBound.compareTo(other.lowerBound) <= 0 && other.upperBound.compareTo(upperBound) <= 0;
+    }
+
+    public boolean isConnected(@NotNull TokenRange other)
+    {
+        return lowerBound.compareTo(other.upperBound) < 0 && other.lowerBound.compareTo(upperBound) < 0;
+    }
+
+    @NotNull
+    public TokenRange intersection(@NotNull TokenRange other)
+    {
+        return new TokenRange(lowerBound.max(other.lowerBound), upperBound.min(other.upperBound));
+    }
+
+    @NotNull
+    public TokenRange span(@NotNull TokenRange other)
+    {
+        return new TokenRange(lowerBound.min(other.lowerBound), upperBound.max(other.upperBound));
+    }
+
+    @Override
+    public boolean equals(@Nullable Object other)
+    {
+        return other instanceof TokenRange
+            && this.lowerBound.equals(((TokenRange) other).lowerBound)
+            && this.upperBound.equals(((TokenRange) other).upperBound);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return lowerBound.hashCode() ^ upperBound.hashCode();
+    }
+
+    @Override
+    @NotNull
+    public String toString()
+    {
+        return lowerBound + "‥" + upperBound;
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/cdc/CommitLog.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/cdc/CommitLog.java
new file mode 100644
index 0000000..f599bba
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/cdc/CommitLog.java
@@ -0,0 +1,176 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.cassandra.spark.cdc;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.lang.builder.EqualsBuilder;
+import org.apache.commons.lang.builder.HashCodeBuilder;
+
+import org.apache.cassandra.spark.data.SSTable;
+import org.apache.cassandra.spark.data.partitioner.CassandraInstance;
+import org.apache.cassandra.spark.stats.Stats;
+import org.apache.cassandra.spark.utils.streaming.SSTableSource;
+import org.jetbrains.annotations.NotNull;
+
+public interface CommitLog extends AutoCloseable
+{
+    /**
+     * @return filename of the CommitLog
+     */
+    String name();
+
+    /**
+     * @return path to the CommitLog
+     */
+    String path();
+
+    /**
+     * @return the max offset that can be read in the CommitLog.
+     *         This may be less than or equal to {@link CommitLog#length()}.
+     *         The reader should not read passed this point when the CommitLog is incomplete.
+     */
+    long maxOffset();
+
+    /**
+     * @return length of the CommitLog in bytes
+     */
+    long length();
+
+    /**
+     * @return an SSTableSource for asynchronously reading the CommitLog bytes
+     */
+    SSTableSource<? extends SSTable> source();
+
+    /**
+     * @return the CassandraInstance this CommitLog resides on
+     */
+    CassandraInstance instance();
+
+    default CommitLog.Marker zeroMarker()
+    {
+        return markerAt(0, 0);
+    }
+
+    default CommitLog.Marker markerAt(long section, int offset)
+    {
+        return new CommitLog.Marker(instance(), section, offset);
+    }
+
+    /**
+     * Override to provide custom stats implementation
+     *
+     * @return stats instance for publishing stats
+     */
+    default Stats stats()
+    {
+        return Stats.DoNothingStats.INSTANCE;
+    }
+
+    class Marker implements Comparable<Marker>
+    {
+        CassandraInstance instance;
+        long segmentId;
+        int position;
+
+        public Marker(CassandraInstance instance, long segmentId, int position)
+        {
+            this.instance = instance;
+            this.segmentId = segmentId;
+            this.position = position;
+        }
+
+        /**
+         * Marks the start position of the section
+         *
+         * @return position in CommitLog of the section
+         */
+        public long segmentId()
+        {
+            return segmentId;
+        }
+
+        public CassandraInstance instance()
+        {
+            return instance;
+        }
+
+        /**
+         * The offset into the section where the mutation starts
+         *
+         * @return mutation offset within the section
+         */
+        public int position()
+        {
+            return position;
+        }
+
+        @Override
+        public int compareTo(@NotNull Marker that)
+        {
+            Preconditions.checkArgument(this.instance.equals(that.instance),
+                                        "CommitLog Markers should be on the same instance");
+            int comparison = Long.compare(this.segmentId, that.segmentId());
+            if (comparison == 0)
+            {
+                return Integer.compare(this.position, that.position());
+            }
+            return comparison;
+        }
+
+        public String toString()
+        {
+            return String.format("{\"segmentId\": %d, \"position\": %d}", segmentId, position);
+        }
+
+        @Override
+        public int hashCode()
+        {
+            return new HashCodeBuilder()
+                   .append(instance)
+                   .append(segmentId)
+                   .append(position)
+                   .toHashCode();
+        }
+
+        @Override
+        public boolean equals(Object other)
+        {
+            if (other == null)
+            {
+                return false;
+            }
+            if (this == other)
+            {
+                return true;
+            }
+            if (this.getClass() != other.getClass())
+            {
+                return false;
+            }
+
+            Marker that = (Marker) other;
+            return new EqualsBuilder()
+                   .append(this.instance, that.instance)
+                   .append(this.segmentId, that.segmentId)
+                   .append(this.position, that.position)
+                   .isEquals();
+        }
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/cdc/CommitLogProvider.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/cdc/CommitLogProvider.java
new file mode 100644
index 0000000..4465d2a
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/cdc/CommitLogProvider.java
@@ -0,0 +1,27 @@
+/*
+ * 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.cassandra.spark.cdc;
+
+import java.util.stream.Stream;
+
+public interface CommitLogProvider
+{
+    Stream<CommitLog> logs();
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/cdc/IPartitionUpdateWrapper.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/cdc/IPartitionUpdateWrapper.java
new file mode 100644
index 0000000..1fe05e2
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/cdc/IPartitionUpdateWrapper.java
@@ -0,0 +1,27 @@
+/*
+ * 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.cassandra.spark.cdc;
+
+public interface IPartitionUpdateWrapper
+{
+    long maxTimestampMicros();
+
+    int dataSize();
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/cdc/TableIdLookup.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/cdc/TableIdLookup.java
new file mode 100644
index 0000000..3c4c532
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/cdc/TableIdLookup.java
@@ -0,0 +1,42 @@
+/*
+ * 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.cassandra.spark.cdc;
+
+import java.util.NoSuchElementException;
+import java.util.UUID;
+
+import org.jetbrains.annotations.Nullable;
+
+public interface TableIdLookup
+{
+    /**
+     * For CDC, the TableId is serialized in the CommitLog, but we generate a new schema instance
+     * in the Spark JVM with a random tableId. We need to convert the tableId used in Spark to match
+     * the tableId used in the Cassandra cluster from the CommitLog to table id we use in Spark.
+     *
+     * @param keyspace keyspace name
+     * @param table    table name
+     * @return tableId used in the Cassandra cluster as UUID, this tableId should match the tableId
+     *         serialized in CommitLog. The return value can be null if the lookup is a no-op.
+     * @throws NoSuchElementException if the tableId cannot be found
+     */
+    @Nullable
+    UUID lookup(String keyspace, String table) throws NoSuchElementException;
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/cdc/watermarker/DoNothingWatermarker.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/cdc/watermarker/DoNothingWatermarker.java
new file mode 100644
index 0000000..21d4ab1
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/cdc/watermarker/DoNothingWatermarker.java
@@ -0,0 +1,83 @@
+/*
+ * 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.cassandra.spark.cdc.watermarker;
+
+import org.apache.cassandra.spark.cdc.CommitLog;
+import org.apache.cassandra.spark.cdc.IPartitionUpdateWrapper;
+import org.apache.cassandra.spark.data.partitioner.CassandraInstance;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Watermarker that does nothing
+ */
+public class DoNothingWatermarker implements Watermarker
+{
+    public static final DoNothingWatermarker INSTANCE = new DoNothingWatermarker();
+
+    @Override
+    public Watermarker instance(String jobId)
+    {
+        return this;
+    }
+
+    @Override
+    public void recordReplicaCount(IPartitionUpdateWrapper update, int numReplicas)
+    {
+    }
+
+    @Override
+    public int replicaCount(IPartitionUpdateWrapper update)
+    {
+        return 0;
+    }
+
+    @Override
+    public void untrackReplicaCount(IPartitionUpdateWrapper update)
+    {
+    }
+
+    @Override
+    public boolean seenBefore(IPartitionUpdateWrapper update)
+    {
+        return false;
+    }
+
+    @Override
+    public void updateHighWaterMark(CommitLog.Marker marker)
+    {
+    }
+
+    @Override
+    @Nullable
+    public CommitLog.Marker highWaterMark(CassandraInstance instance)
+    {
+        return null;
+    }
+
+    @Override
+    public void persist(@Nullable Long maxAgeMicros)
+    {
+    }
+
+    @Override
+    public void clear()
+    {
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/cdc/watermarker/Watermarker.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/cdc/watermarker/Watermarker.java
new file mode 100644
index 0000000..d49fe69
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/cdc/watermarker/Watermarker.java
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.cassandra.spark.cdc.watermarker;
+
+import org.apache.cassandra.spark.cdc.CommitLog;
+import org.apache.cassandra.spark.cdc.IPartitionUpdateWrapper;
+import org.apache.cassandra.spark.data.partitioner.CassandraInstance;
+import org.jetbrains.annotations.Nullable;
+
+public interface Watermarker
+{
+    /**
+     * @param jobId job id that uniquely tracks this Spark job
+     * @return a Watermarker for this Spark job
+     */
+    Watermarker instance(String jobId);
+
+    /**
+     * Insufficient replica copies to publish update, so record number of replicas read so we can publish once we achieve Consistency Level
+     *
+     * @param update the CDC update we need to track until we receive CL mutations
+     */
+    void recordReplicaCount(IPartitionUpdateWrapper update, int numReplicas);
+
+    /**
+     * Return how many replicas we have previously read for this mutation
+     *
+     * @param update the CDC update
+     * @return number of replicas previously received or 0 if never seen before
+     */
+    int replicaCount(IPartitionUpdateWrapper update);
+
+    /**
+     * We received sufficient replica copies for a given update we can stop tracking the number of replicas for this update
+     *
+     * @param update the CDC update
+     */
+    void untrackReplicaCount(IPartitionUpdateWrapper update);
+
+    /**
+     * @param update the CDC update
+     * @return true if we have previously seen this update before
+     */
+    boolean seenBefore(IPartitionUpdateWrapper update);
+
+    /**
+     * Marks how far we have read in the CommitLogs for a particular Cassandra instance.
+     * We start reading from this mark on the next round.
+     *
+     * @param marker highwater mark
+     */
+    void updateHighWaterMark(CommitLog.Marker marker);
+
+    /**
+     * @param instance Cassandra instance
+     * @return the previous high water mark for a given Cassandra instance
+     */
+    @Nullable
+    CommitLog.Marker highWaterMark(CassandraInstance instance);
+
+    /**
+     * Persist watermark state to a persistent external store that can be resumed in the next Spark Streaming batch
+     *
+     * @param maxAgeMicros the maximum age in microseconds
+     */
+    void persist(@Nullable Long maxAgeMicros);
+
+    /**
+     * Clear watermark history
+     */
+    void clear();
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/config/SchemaFeature.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/config/SchemaFeature.java
new file mode 100644
index 0000000..30b03f7
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/config/SchemaFeature.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.cassandra.spark.config;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.cassandra.spark.data.CqlTable;
+import org.apache.cassandra.spark.sparksql.RowBuilder;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * Defines the properties of schema features. It requires to be implemented as enum
+ */
+public interface SchemaFeature
+{
+    /**
+     * The {@link DataType} of the field
+     *
+     * @return DataType
+     */
+    DataType fieldDataType();
+
+    /**
+     * Generate a dynamic {@link DataType} based on {@link CqlTable} and the {@link StructType} Spark schema.
+     * If a feature has a fixed {@link DataType}, the method does not need to be overridden.
+     *
+     * @param table    the CQL table schema
+     * @param sparkSchema the Spark schema
+     */
+    default void generateDataType(CqlTable table, StructType sparkSchema)
+    {
+        // Do nothing
+    }
+
+    /**
+     * Decorate the Spark row builder according to the feature
+     *
+     * @param builder the row builder
+     * @return a new decorated builder
+     */
+    RowBuilder decorate(RowBuilder builder);
+
+    /**
+     * The option name used in the Spark options
+     *
+     * @return option name
+     */
+    default String optionName()
+    {
+        return fieldName();
+    }
+
+    /**
+     * The Spark {@link StructField} according to the feature
+     *
+     * @return struct field
+     */
+    default StructField field()
+    {
+        return new StructField(fieldName(), fieldDataType(), fieldNullable(), fieldMetadata());
+    }
+
+    /**
+     * The name of the field
+     *
+     * @return field name
+     */
+    default String fieldName()
+    {
+        Preconditions.checkState(this instanceof Enum<?>, "Only implement this interface in enum");
+        Enum<?> e = (Enum<?>) this;
+        return e.name().toLowerCase();
+    }
+
+    /**
+     * Define if the field is nullable
+     *
+     * @return true, if the field is nullable
+     */
+    default boolean fieldNullable()
+    {
+        return true;
+    }
+
+    /**
+     * The metadata used for the field
+     *
+     * @return metadata
+     */
+    default Metadata fieldMetadata()
+    {
+        return Metadata.empty();
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/config/SchemaFeatureSet.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/config/SchemaFeatureSet.java
new file mode 100644
index 0000000..61bbe35
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/config/SchemaFeatureSet.java
@@ -0,0 +1,182 @@
+/*
+ * 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.cassandra.spark.config;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+import org.apache.cassandra.spark.data.CqlTable;
+import org.apache.cassandra.spark.sparksql.CellTombstonesInComplexDecorator;
+import org.apache.cassandra.spark.sparksql.LastModifiedTimestampDecorator;
+import org.apache.cassandra.spark.sparksql.RangeTombstoneDecorator;
+import org.apache.cassandra.spark.sparksql.RangeTombstoneMarker;
+import org.apache.cassandra.spark.sparksql.RowBuilder;
+import org.apache.cassandra.spark.sparksql.UpdateFlagDecorator;
+import org.apache.cassandra.spark.sparksql.UpdatedFieldsIndicatorDecorator;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+public enum SchemaFeatureSet implements SchemaFeature
+{
+    // NOTE: The order matters!
+
+    // Special column that passes over last modified timestamp for a row
+    LAST_MODIFIED_TIMESTAMP
+        {
+            @Override
+            public DataType fieldDataType()
+            {
+                return DataTypes.TimestampType;
+            }
+
+            @Override
+            public RowBuilder decorate(RowBuilder builder)
+            {
+                return new LastModifiedTimestampDecorator(builder, fieldName());
+            }
+        },
+
+    // Special column that passes over updated bitset field
+    // indicating which columns are unset and which are tombstones this is only used for CDC
+    UPDATED_FIELDS_INDICATOR
+        {
+            @Override
+            public DataType fieldDataType()
+            {
+                return DataTypes.BinaryType;
+            }
+
+            @Override
+            public RowBuilder decorate(RowBuilder builder)
+            {
+                return new UpdatedFieldsIndicatorDecorator(builder);
+            }
+        },
+
+    // Special column that passes over boolean field
+    // marking if mutation was an UPDATE or an INSERT this is only used for CDC
+    UPDATE_FLAG
+        {
+            @Override
+            public DataType fieldDataType()
+            {
+                return DataTypes.BooleanType;
+            }
+
+            @Override
+            public RowBuilder decorate(RowBuilder builder)
+            {
+                return new UpdateFlagDecorator(builder);
+            }
+        },
+
+    // Feature column that contains the column_name
+    // to a list of keys of the tombstoned values in a complex data type this is only used for CDC
+    CELL_DELETION_IN_COMPLEX
+        {
+            @Override
+            public DataType fieldDataType()
+            {
+                return DataTypes.createMapType(DataTypes.StringType, DataTypes.createArrayType(DataTypes.BinaryType));
+            }
+
+            @Override
+            public RowBuilder decorate(RowBuilder builder)
+            {
+                return new CellTombstonesInComplexDecorator(builder);
+            }
+        },
+
+    RANGE_DELETION
+        {
+            private transient DataType dataType;
+
+            @Override
+            public void generateDataType(CqlTable table, StructType sparkSchema)
+            {
+                // When there is no clustering keys, range deletion won't happen;
+                // such deletion applies only when there are clustering key(s)
+                if (table.numClusteringKeys() == 0)
+                {
+                    // Assign a dummy data type, it won't be reach with such CQL schema
+                    dataType = DataTypes.BooleanType;
+                    return;
+                }
+
+                List<StructField> clusteringKeyFields = table.clusteringKeys().stream()
+                                                             .map(cqlField -> sparkSchema.apply(cqlField.name()))
+                                                             .collect(Collectors.toList());
+                StructType clusteringKeys = DataTypes.createStructType(clusteringKeyFields);
+                StructField[] rangeTombstone = new StructField[RangeTombstoneMarker.TOTAL_FIELDS];
+                // The array of binaries follows the same seq of the clustering key definition, e.g. for primary key
+                // (pk, ck1, ck2), the array value could be [ck1] or [ck1, ck2], but never (ck2) without ck1
+                rangeTombstone[RangeTombstoneMarker.START_FIELD_POSITION] =
+                        DataTypes.createStructField("Start", clusteringKeys, true);
+                // Default to be inclusive if null
+                rangeTombstone[RangeTombstoneMarker.START_INCLUSIVE_FIELD_POSITION] =
+                        DataTypes.createStructField("StartInclusive", DataTypes.BooleanType, true);
+                rangeTombstone[RangeTombstoneMarker.END_FIELD_POSITION] =
+                        DataTypes.createStructField("End", clusteringKeys, true);
+                // Default to be inclusive if null
+                rangeTombstone[RangeTombstoneMarker.END_INCLUSIVE_FIELD_POSITION] =
+                        DataTypes.createStructField("EndInclusive", DataTypes.BooleanType, true);
+                dataType = DataTypes.createArrayType(DataTypes.createStructType(rangeTombstone));
+            }
+
+            @Override
+            public DataType fieldDataType()
+            {
+                Preconditions.checkNotNull(dataType, "The dynamic data type is not initialized");
+                return dataType;
+            }
+
+            @Override
+            public RowBuilder decorate(RowBuilder builder)
+            {
+                return new RangeTombstoneDecorator(builder);
+            }
+        };
+
+    public static final List<SchemaFeature> ALL_CDC_FEATURES = ImmutableList.of(UPDATED_FIELDS_INDICATOR,
+                                                                                UPDATE_FLAG,
+                                                                                CELL_DELETION_IN_COMPLEX,
+                                                                                RANGE_DELETION);
+
+    /**
+     * Initialize the requested features from the input options
+     *
+     * @param options
+     * @return the requested features list. If none is requested, an empty list is returned
+     */
+    public static List<SchemaFeature> initializeFromOptions(Map<String, String> options)
+    {
+        // TODO: Some features are only valid for the CDC scenario; probably reject early
+        return Arrays.stream(values())
+                     .filter(feature -> Boolean.parseBoolean(options.getOrDefault(feature.optionName(), "false").toLowerCase()))
+                     .collect(Collectors.toList());
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/data/CqlField.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/data/CqlField.java
new file mode 100644
index 0000000..2d48d78
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/data/CqlField.java
@@ -0,0 +1,465 @@
+/*
+ * 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.cassandra.spark.data;
+
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.commons.lang.builder.EqualsBuilder;
+import org.apache.commons.lang.builder.HashCodeBuilder;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+import org.apache.cassandra.bridge.BigNumberConfig;
+import org.apache.cassandra.bridge.CassandraBridge;
+import org.apache.cassandra.bridge.CassandraVersion;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.jetbrains.annotations.NotNull;
+
+@SuppressWarnings({"WeakerAccess", "unused"})
+public class CqlField implements Serializable, Comparable<CqlField>
+{
+    private static final long serialVersionUID = 42L;
+
+    public interface CqlType extends Serializable, Comparator<Object>
+    {
+        enum InternalType
+        {
+            NativeCql, Set, List, Map, Frozen, Udt, Tuple;
+
+            public static InternalType fromString(String name)
+            {
+                switch (name.toLowerCase())
+                {
+                    case "set":
+                        return Set;
+                    case "list":
+                        return List;
+                    case "map":
+                        return Map;
+                    case "tuple":
+                        return Tuple;
+                    case "udt":
+                        return Udt;
+                    case "frozen":
+                        return Frozen;
+                    default:
+                        return NativeCql;
+                }
+            }
+        }
+
+        boolean isSupported();
+
+        Object toSparkSqlType(Object value);
+
+        Object toSparkSqlType(Object value, boolean isFrozen);
+
+        Object deserialize(ByteBuffer buffer);
+
+        Object deserialize(ByteBuffer buffer, boolean isFrozen);
+
+        ByteBuffer serialize(Object value);
+
+        boolean equals(Object first, Object second);
+
+        CassandraVersion version();
+
+        InternalType internalType();
+
+        String name();
+
+        String cqlName();
+
+        /*
+            SparkSQL      |    Java
+            ByteType      |    byte or Byte
+            ShortType     |    short or Short
+            IntegerType   |    int or Integer
+            LongType      |    long or Long
+            FloatType     |    float or Float
+            DoubleType    |    double or Double
+            DecimalType   |    java.math.BigDecimal
+            StringType    |    String
+            BinaryType    |    byte[]
+            BooleanType   |    boolean or Boolean
+            TimestampType |    java.sql.Timestamp
+            DateType      |    java.sql.Date
+            ArrayType     |    java.util.List
+            MapType       |    java.util.Map
+
+            See: https://spark.apache.org/docs/latest/sql-reference.html
+        */
+        DataType sparkSqlType();
+
+        DataType sparkSqlType(BigNumberConfig bigNumberConfig);
+
+        void write(Output output);
+
+        Set<CqlField.CqlUdt> udts();
+
+        @VisibleForTesting
+        int cardinality(int orElse);
+
+        @VisibleForTesting
+        Object sparkSqlRowValue(GenericInternalRow row, int position);
+
+        @VisibleForTesting
+        Object sparkSqlRowValue(Row row, int position);
+
+        @VisibleForTesting
+        Object randomValue(int minCollectionSize);
+
+        @VisibleForTesting
+        Object toTestRowType(Object value);
+
+        @VisibleForTesting
+        Object convertForCqlWriter(Object value, CassandraVersion version);
+
+        // Kryo Serialization
+
+        static void write(CqlType type, Output out)
+        {
+            out.writeInt(type.internalType().ordinal());
+        }
+
+        static CqlType read(Input input, CassandraBridge bridge)
+        {
+            InternalType internalType = InternalType.values()[input.readInt()];
+            return bridge.readType(internalType, input);
+        }
+    }
+
+    public interface NativeType extends CqlType
+    {
+    }
+
+    public interface CqlCustom extends CqlType
+    {
+        /**
+         * @return the fully qualified name of the subtype of {@code org.apache.cassandra.db.marshal.AbstractType} that
+         * represents this type server-side
+         */
+        String customTypeClassName();
+    }
+
+    public interface CqlCollection extends CqlType
+    {
+        CqlFrozen frozen();
+
+        List<CqlType> types();
+
+        CqlField.CqlType type();
+
+        CqlField.CqlType type(int position);
+    }
+
+    public interface CqlMap extends CqlCollection
+    {
+        CqlField.CqlType keyType();
+
+        CqlField.CqlType valueType();
+    }
+
+    public interface CqlSet extends CqlCollection
+    {
+    }
+
+    public interface CqlList extends CqlCollection
+    {
+    }
+
+    public interface CqlTuple extends CqlCollection
+    {
+        ByteBuffer serializeTuple(Object[] values);
+
+        Object[] deserializeTuple(ByteBuffer buffer, boolean isFrozen);
+    }
+
+    public interface CqlFrozen extends CqlType
+    {
+        CqlField.CqlType inner();
+    }
+
+    public interface CqlUdt extends CqlType
+    {
+        CqlFrozen frozen();
+
+        String createStatement(String keyspace);
+
+        String keyspace();
+
+        List<CqlField> fields();
+
+        CqlField field(String name);
+
+        CqlField field(int position);
+
+        ByteBuffer serializeUdt(Map<String, Object> values);
+
+        Map<String, Object> deserializeUdt(ByteBuffer buffer, boolean isFrozen);
+    }
+
+    public interface CqlUdtBuilder
+    {
+        CqlUdtBuilder withField(String name, CqlField.CqlType type);
+
+        CqlField.CqlUdt build();
+    }
+
+    public enum SortOrder
+    {
+        ASC,
+        DESC
+    }
+
+    private final String name;
+    private final boolean isPartitionKey;
+    private final boolean isClusteringColumn;
+    private final boolean isStaticColumn;
+    private final CqlType type;
+    private final int position;
+
+    public CqlField(boolean isPartitionKey,
+                    boolean isClusteringColumn,
+                    boolean isStaticColumn,
+                    String name,
+                    CqlType type,
+                    int position)
+    {
+        Preconditions.checkArgument(!(isPartitionKey && isClusteringColumn),
+                                    "Field cannot be both partition key and clustering key");
+        Preconditions.checkArgument(!(isPartitionKey && isStaticColumn),
+                                    "Field cannot be both partition key and static column");
+        Preconditions.checkArgument(!(isClusteringColumn && isStaticColumn),
+                                    "Field cannot be both clustering key and static column");
+        this.isPartitionKey = isPartitionKey;
+        this.isClusteringColumn = isClusteringColumn;
+        this.isStaticColumn = isStaticColumn;
+        this.name = name.replaceAll("\"", "");
+        this.type = type;
+        this.position = position;
+    }
+
+    public boolean isPartitionKey()
+    {
+        return isPartitionKey;
+    }
+
+    public boolean isPrimaryKey()
+    {
+        return isPartitionKey || isClusteringColumn;
+    }
+
+    public boolean isClusteringColumn()
+    {
+        return isClusteringColumn;
+    }
+
+    public boolean isStaticColumn()
+    {
+        return isStaticColumn;
+    }
+
+    public boolean isValueColumn()
+    {
+        return !isPartitionKey && !isClusteringColumn && !isStaticColumn;
+    }
+
+    public boolean isNonValueColumn()
+    {
+        return !isValueColumn();
+    }
+
+    public String name()
+    {
+        return name;
+    }
+
+    public CqlType type()
+    {
+        return type;
+    }
+
+    public Object deserialize(ByteBuffer buffer)
+    {
+        return deserialize(buffer, false);
+    }
+
+    public Object deserialize(ByteBuffer buffer, boolean isFrozen)
+    {
+        return type().deserialize(buffer, isFrozen);
+    }
+
+    public ByteBuffer serialize(Object value)
+    {
+        return type.serialize(value);
+    }
+
+    public String cqlTypeName()
+    {
+        return type.cqlName();
+    }
+
+    public int position()
+    {
+        return position;
+    }
+
+    @VisibleForTesting
+    public CqlField cloneWithPosition(int position)
+    {
+        return new CqlField(isPartitionKey, isClusteringColumn, isStaticColumn, name, type, position);
+    }
+
+    @Override
+    public String toString()
+    {
+        return name + " (" + type + ")";
+    }
+
+    @Override
+    public int compareTo(@NotNull CqlField that)
+    {
+        return Integer.compare(this.position, that.position);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return new HashCodeBuilder()
+               .append(name)
+               .append(isPartitionKey)
+               .append(isClusteringColumn)
+               .append(isStaticColumn)
+               .append(type)
+               .append(position)
+               .toHashCode();
+    }
+
+    @Override
+    public boolean equals(Object other)
+    {
+        if (other == null)
+        {
+            return false;
+        }
+        if (this == other)
+        {
+            return true;
+        }
+        if (this.getClass() != other.getClass())
+        {
+            return false;
+        }
+
+        CqlField that = (CqlField) other;
+        return new EqualsBuilder()
+               .append(this.name, that.name)
+               .append(this.isPartitionKey, that.isPartitionKey)
+               .append(this.isClusteringColumn, that.isClusteringColumn)
+               .append(this.isStaticColumn, that.isStaticColumn)
+               .append(this.type, that.type)
+               .append(this.position, that.position)
+               .isEquals();
+    }
+
+    public boolean equals(Object first, Object second)
+    {
+        return type().equals(first, second);
+    }
+
+    public static boolean equalsArrays(Object[] first, Object[] second, Function<Integer, CqlType> types)
+    {
+        for (int index = 0; index < Math.min(first.length, second.length); index++)
+        {
+            if (!types.apply(index).equals(first[index], second[index]))
+            {
+                return false;
+            }
+        }
+        return first.length == second.length;
+    }
+
+    public int compare(Object first, Object second)
+    {
+        return type().compare(first, second);
+    }
+
+    public static int compareArrays(Object[] first, Object[] second, Function<Integer, CqlType> types)
+    {
+        for (int index = 0; index < Math.min(first.length, second.length); index++)
+        {
+            int comparison = types.apply(index).compare(first[index], second[index]);
+            if (comparison != 0)
+            {
+                return comparison;
+            }
+        }
+        return Integer.compare(first.length, second.length);
+    }
+
+    public static class Serializer extends com.esotericsoftware.kryo.Serializer<CqlField>
+    {
+        private final CassandraBridge bridge;
+
+        public Serializer(CassandraBridge bridge)
+        {
+            this.bridge = bridge;
+        }
+
+        @Override
+        public CqlField read(Kryo kryo, Input input, Class type)
+        {
+            return new CqlField(input.readBoolean(),
+                                input.readBoolean(),
+                                input.readBoolean(),
+                                input.readString(),
+                                CqlType.read(input, bridge),
+                                input.readInt());
+        }
+
+        @Override
+        public void write(Kryo kryo, Output output, CqlField field)
+        {
+            output.writeBoolean(field.isPartitionKey());
+            output.writeBoolean(field.isClusteringColumn());
+            output.writeBoolean(field.isStaticColumn());
+            output.writeString(field.name());
+            field.type().write(output);
+            output.writeInt(field.position());
+        }
+    }
+
+    public static UnsupportedOperationException notImplemented(CqlType type)
+    {
+        return new UnsupportedOperationException(type.toString() + " type not implemented");
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/data/CqlTable.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/data/CqlTable.java
new file mode 100644
index 0000000..bf8a180
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/data/CqlTable.java
@@ -0,0 +1,329 @@
+/*
+ * 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.cassandra.spark.data;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import org.apache.commons.lang.builder.EqualsBuilder;
+import org.apache.commons.lang.builder.HashCodeBuilder;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+import org.apache.cassandra.bridge.CassandraBridge;
+import org.jetbrains.annotations.NotNull;
+
+@SuppressWarnings({ "WeakerAccess", "unused" })
+public class CqlTable implements Serializable
+{
+    public static final long serialVersionUID = 42L;
+
+    private final ReplicationFactor replicationFactor;
+    private final String keyspace;
+    private final String table;
+    private final String createStatement;
+    private final List<CqlField> fields;
+    private final Set<CqlField.CqlUdt> udts;
+
+    private final Map<String, CqlField> fieldsMap;
+    private final List<CqlField> partitionKeys;
+    private final List<CqlField> clusteringKeys;
+    private final List<CqlField> staticColumns;
+    private final List<CqlField> valueColumns;
+    private final transient Map<String, CqlField> columns;
+    private final int indexCount;
+
+    public CqlTable(@NotNull String keyspace,
+                    @NotNull String table,
+                    @NotNull String createStatement,
+                    @NotNull ReplicationFactor replicationFactor,
+                    @NotNull List<CqlField> fields)
+    {
+        this(keyspace, table, createStatement, replicationFactor, fields, Collections.emptySet(), 0);
+    }
+
+    public CqlTable(@NotNull String keyspace,
+                    @NotNull String table,
+                    @NotNull String createStatement,
+                    @NotNull ReplicationFactor replicationFactor,
+                    @NotNull List<CqlField> fields,
+                    @NotNull Set<CqlField.CqlUdt> udts,
+                    int indexCount)
+    {
+        this.keyspace = keyspace;
+        this.table = table;
+        this.createStatement = createStatement;
+        this.replicationFactor = replicationFactor;
+        this.fields = fields.stream().sorted().collect(Collectors.toList());
+        this.fieldsMap = this.fields.stream().collect(Collectors.toMap(CqlField::name, Function.identity()));
+        this.partitionKeys = this.fields.stream().filter(CqlField::isPartitionKey).sorted().collect(Collectors.toList());
+        this.clusteringKeys = this.fields.stream().filter(CqlField::isClusteringColumn).sorted().collect(Collectors.toList());
+        this.staticColumns = this.fields.stream().filter(CqlField::isStaticColumn).sorted().collect(Collectors.toList());
+        this.valueColumns = this.fields.stream().filter(CqlField::isValueColumn).sorted().collect(Collectors.toList());
+        this.udts = Collections.unmodifiableSet(udts);
+        this.indexCount = indexCount;
+
+        // We use a linked hashmap to guarantee ordering of a 'SELECT * FROM ...'
+        this.columns = new LinkedHashMap<>();
+        for (CqlField column : partitionKeys)
+        {
+            columns.put(column.name(), column);
+        }
+        for (CqlField column : clusteringKeys)
+        {
+            columns.put(column.name(), column);
+        }
+        for (CqlField column : staticColumns)
+        {
+            columns.put(column.name(), column);
+        }
+        for (CqlField column : valueColumns)
+        {
+            columns.put(column.name(), column);
+        }
+    }
+
+    public ReplicationFactor replicationFactor()
+    {
+        return replicationFactor;
+    }
+
+    public CqlField column(String columnName)
+    {
+        return columns.get(columnName);
+    }
+
+    public List<CqlField> columns()
+    {
+        return new ArrayList<>(columns.values());
+    }
+
+    public List<CqlField> primaryKey()
+    {
+        List<CqlField> pk = new ArrayList<>(partitionKeys.size() + clusteringKeys.size());
+        pk.addAll(partitionKeys);
+        pk.addAll(clusteringKeys);
+        return pk;
+    }
+
+    public List<CqlField> partitionKeys()
+    {
+        return partitionKeys;
+    }
+
+    public int numPartitionKeys()
+    {
+        return partitionKeys.size();
+    }
+
+    public List<CqlField> clusteringKeys()
+    {
+        return clusteringKeys;
+    }
+
+    public int numClusteringKeys()
+    {
+        return clusteringKeys.size();
+    }
+
+    public int numPrimaryKeyColumns()
+    {
+        return numPartitionKeys() + numClusteringKeys();
+    }
+
+    public int numNonValueColumns()
+    {
+        return numPartitionKeys() + numClusteringKeys() + numStaticColumns();
+    }
+
+    public List<CqlField> valueColumns()
+    {
+        return valueColumns;
+    }
+
+    public int numValueColumns()
+    {
+        return valueColumns.size();
+    }
+
+    public List<CqlField> staticColumns()
+    {
+        return staticColumns;
+    }
+
+    public int numStaticColumns()
+    {
+        return staticColumns.size();
+    }
+
+    public int numFields()
+    {
+        return fields.size();
+    }
+
+    public boolean has(String field)
+    {
+        return fieldsMap.containsKey(field);
+    }
+
+    public List<CqlField> fields()
+    {
+        return fields;
+    }
+
+    public Set<CqlField.CqlUdt> udts()
+    {
+        return udts;
+    }
+
+    public Set<String> udtCreateStmts()
+    {
+        return udts.stream()
+                   .map(udt -> udt.createStatement(keyspace))
+                   .collect(Collectors.toSet());
+    }
+
+    public CqlField getField(String name)
+    {
+        return fieldsMap.get(name);
+    }
+
+    public String keyspace()
+    {
+        return keyspace;
+    }
+
+    public String table()
+    {
+        return table;
+    }
+
+    public String createStatement()
+    {
+        return createStatement;
+    }
+
+    public int indexCount()
+    {
+        return indexCount;
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return new HashCodeBuilder()
+               .append(keyspace)
+               .append(table)
+               .append(createStatement)
+               .append(fields)
+               .append(udts)
+               .toHashCode();
+    }
+
+    @Override
+    public boolean equals(Object other)
+    {
+        if (other == null)
+        {
+            return false;
+        }
+        if (this == other)
+        {
+            return true;
+        }
+        if (this.getClass() != other.getClass())
+        {
+            return false;
+        }
+
+        CqlTable that = (CqlTable) other;
+        return new EqualsBuilder()
+               .append(this.keyspace, that.keyspace)
+               .append(this.table, that.table)
+               .append(this.createStatement, that.createStatement)
+               .append(this.fields, that.fields)
+               .append(this.udts, that.udts)
+               .isEquals();
+    }
+
+    public static class Serializer extends com.esotericsoftware.kryo.Serializer<CqlTable>
+    {
+        private final CassandraBridge bridge;
+
+        public Serializer(CassandraBridge bridge)
+        {
+            this.bridge = bridge;
+        }
+
+        @Override
+        public CqlTable read(Kryo kryo, Input input, Class type)
+        {
+            String keyspace = input.readString();
+            String table = input.readString();
+            String createStatement = input.readString();
+            ReplicationFactor replicationFactor = kryo.readObject(input, ReplicationFactor.class);
+            int numFields = input.readInt();
+            List<CqlField> fields = new ArrayList<>(numFields);
+            for (int field = 0; field < numFields; field++)
+            {
+                fields.add(kryo.readObject(input, CqlField.class));
+            }
+            int numUdts = input.readInt();
+            Set<CqlField.CqlUdt> udts = new LinkedHashSet<>(numUdts);
+            for (int udt = 0; udt < numUdts; udt++)
+            {
+                udts.add((CqlField.CqlUdt) CqlField.CqlType.read(input, bridge));
+            }
+            int indexCount = input.readInt();
+            return new CqlTable(keyspace, table, createStatement, replicationFactor, fields, udts, indexCount);
+        }
+
+        @Override
+        public void write(Kryo kryo, Output output, CqlTable table)
+        {
+            output.writeString(table.keyspace());
+            output.writeString(table.table());
+            output.writeString(table.createStatement());
+            kryo.writeObject(output, table.replicationFactor());
+            List<CqlField> fields = table.fields();
+            output.writeInt(fields.size());
+            for (CqlField field : fields)
+            {
+                kryo.writeObject(output, field);
+            }
+            Set<CqlField.CqlUdt> udts = table.udts();
+            output.writeInt(udts.size());
+            for (CqlField.CqlUdt udt : udts)
+            {
+                udt.write(output);
+            }
+            output.writeInt(table.indexCount());
+        }
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/data/DataLayer.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/data/DataLayer.java
new file mode 100644
index 0000000..b2cd8c0
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/data/DataLayer.java
@@ -0,0 +1,356 @@
+/*
+ * 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.cassandra.spark.data;
+
+import java.io.Serializable;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import java.util.stream.Collectors;
+
+import org.apache.commons.lang.StringUtils;
+
+import org.apache.cassandra.bridge.BigNumberConfig;
+import org.apache.cassandra.bridge.CassandraBridge;
+import org.apache.cassandra.bridge.CassandraVersion;
+import org.apache.cassandra.spark.cdc.CommitLogProvider;
+import org.apache.cassandra.spark.cdc.TableIdLookup;
+import org.apache.cassandra.spark.cdc.watermarker.DoNothingWatermarker;
+import org.apache.cassandra.spark.cdc.watermarker.Watermarker;
+import org.apache.cassandra.spark.config.SchemaFeature;
+import org.apache.cassandra.spark.data.partitioner.Partitioner;
+import org.apache.cassandra.spark.reader.EmptyStreamScanner;
+import org.apache.cassandra.spark.reader.StreamScanner;
+import org.apache.cassandra.spark.sparksql.NoMatchFoundException;
+import org.apache.cassandra.spark.sparksql.filters.CdcOffsetFilter;
+import org.apache.cassandra.spark.sparksql.filters.PartitionKeyFilter;
+import org.apache.cassandra.spark.sparksql.filters.PruneColumnFilter;
+import org.apache.cassandra.spark.sparksql.filters.SparkRangeFilter;
+import org.apache.cassandra.spark.stats.Stats;
+import org.apache.cassandra.spark.utils.TimeProvider;
+import org.apache.spark.sql.sources.EqualTo;
+import org.apache.spark.sql.sources.Filter;
+import org.apache.spark.sql.sources.In;
+import org.apache.spark.sql.types.MetadataBuilder;
+import org.apache.spark.sql.types.StructType;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+@SuppressWarnings({ "unused", "WeakerAccess" })
+public abstract class DataLayer implements Serializable
+{
+    public static final long serialVersionUID = 42L;
+
+    public DataLayer()
+    {
+    }
+
+    /**
+     * Map Cassandra CQL table schema to SparkSQL StructType
+     *
+     * @return StructType representation of CQL table
+     */
+    public StructType structType()
+    {
+        StructType structType = new StructType();
+        for (CqlField field : cqlTable().fields())
+        {
+            // Pass Cassandra field metadata in StructField metadata
+            MetadataBuilder metadata = new MetadataBuilder();
+            metadata.putLong("position", field.position());
+            metadata.putString("cqlType", field.cqlTypeName());
+            metadata.putBoolean("isPartitionKey", field.isPartitionKey());
+            metadata.putBoolean("isPrimaryKey", field.isPrimaryKey());
+            metadata.putBoolean("isClusteringKey", field.isClusteringColumn());
+            metadata.putBoolean("isStaticColumn", field.isStaticColumn());
+            metadata.putBoolean("isValueColumn", field.isValueColumn());
+
+            structType = structType.add(field.name(),
+                                        field.type().sparkSqlType(bigNumberConfig(field)),
+                                        true,
+                                        metadata.build());
+        }
+
+        // Append the requested feature fields
+        for (SchemaFeature feature : requestedFeatures())
+        {
+            feature.generateDataType(cqlTable(), structType);
+            structType = structType.add(feature.field());
+        }
+
+        return structType;
+    }
+
+    public List<SchemaFeature> requestedFeatures()
+    {
+        return Collections.emptyList();
+    }
+
+    /**
+     * DataLayer can override this method to return the BigInteger/BigDecimal precision/scale values for a given column
+     *
+     * @param field the CQL field
+     * @return a BigNumberConfig object that specifies the desired precision/scale for BigDecimal and BigInteger
+     */
+    public BigNumberConfig bigNumberConfig(CqlField field)
+    {
+        return BigNumberConfig.DEFAULT;
+    }
+
+    /**
+     * @return Cassandra version (3.0, 4.0 etc)
+     */
+    public CassandraVersion version()
+    {
+        return bridge().getVersion();
+    }
+
+    /**
+     * @return version-specific CassandraBridge wrapping shaded packages
+     */
+    public abstract CassandraBridge bridge();
+
+    public abstract int partitionCount();
+
+    /**
+     * @return CqlTable object for table being read, batch/bulk read jobs only
+     */
+    public abstract CqlTable cqlTable();
+
+    public abstract boolean isInPartition(int partitionId, BigInteger token, ByteBuffer key);
+
+    public List<PartitionKeyFilter> partitionKeyFiltersInRange(
+            int partitionId,
+            List<PartitionKeyFilter> partitionKeyFilters) throws NoMatchFoundException
+    {
+        return partitionKeyFilters;
+    }
+
+    public abstract CommitLogProvider commitLogs(int partitionId);
+
+    public abstract TableIdLookup tableIdLookup();
+
+    /**
+     * DataLayer implementation should provide a SparkRangeFilter to filter out partitions and mutations
+     * that do not overlap with the Spark worker's token range
+     *
+     * @param partitionId the partitionId for the task
+     * @return SparkRangeFilter for the Spark worker's token range
+     */
+    public SparkRangeFilter sparkRangeFilter(int partitionId)
+    {
+        return null;
+    }
+
+    /**
+     * DataLayer implementation should provide an ExecutorService for doing blocking I/O
+     * when opening SSTable readers or reading CDC CommitLogs.
+     * It is the responsibility of the DataLayer implementation to appropriately size and manage this ExecutorService.
+     *
+     * @return executor service
+     */
+    protected abstract ExecutorService executorService();
+
+    /**
+     * @param partitionId         the partitionId of the task
+     * @param sparkRangeFilter    spark range filter
+     * @param partitionKeyFilters the list of partition key filters
+     * @return set of SSTables
+     */
+    public abstract SSTablesSupplier sstables(int partitionId,
+                                              @Nullable SparkRangeFilter sparkRangeFilter,
+                                              @NotNull List<PartitionKeyFilter> partitionKeyFilters);
+
+    public abstract Partitioner partitioner();
+
+    /**
+     * It specifies the minimum number of replicas required for CDC.
+     * For example, the minimum number of PartitionUpdates for compaction,
+     * and the minimum number of replicas to pull logs from to proceed to compaction.
+     *
+     * @return the minimum number of replicas. The returned value must be 1 or more.
+     */
+    public int minimumReplicasForCdc()
+    {
+        return 1;
+    }
+
+    /**
+     * @return a string that uniquely identifies this Spark job
+     */
+    public abstract String jobId();
+
+    /**
+     * Override this method with a Watermarker implementation
+     * that persists high and low watermarks per Spark partition between Streaming batches
+     *
+     * @return watermarker for persisting high and low watermark and late updates
+     */
+    public Watermarker cdcWatermarker()
+    {
+        return DoNothingWatermarker.INSTANCE;
+    }
+
+    public Duration cdcWatermarkWindow()
+    {
+        return Duration.ofSeconds(30);
+    }
+
+    public StreamScanner openCdcScanner(int partitionId, @Nullable CdcOffsetFilter offset)
+    {
+        return bridge().getCdcScanner(partitionId,
+                                      cqlTable(),
+                                      partitioner(),
+                                      commitLogs(partitionId),
+                                      tableIdLookup(),
+                                      stats(),
+                                      sparkRangeFilter(partitionId),
+                                      offset,
+                                      minimumReplicasForCdc(),
+                                      cdcWatermarker(),
+                                      jobId(),
+                                      executorService(),
+                                      timeProvider());
+    }
+
+    public StreamScanner openCompactionScanner(int partitionId, List<PartitionKeyFilter> partitionKeyFilters)
+    {
+        return openCompactionScanner(partitionId, partitionKeyFilters, null);
+    }
+
+    /**
+     * When true the SSTableReader should attempt to find the offset into the Data.db file for the Spark worker's
+     * token range. This works by first binary searching the Summary.db file to find offset into Index.db file,
+     * then reading the Index.db from the Summary.db offset to find the first offset in the Data.db file
+     * that overlaps with the Spark worker's token range. This enables the reader to start reading from the first
+     * in-range partition in the Data.db file, and close after reading the last partition. This feature improves
+     * scalability as more Spark workers shard the token range into smaller subranges. This avoids wastefully reading
+     * the Data.db file for out-of-range partitions.
+     *
+     * @return true if, the SSTableReader should attempt to read Summary.db and Index.db files
+     *         to find the start index offset into the Data.db file that overlaps with the Spark workers token range
+     */
+    public boolean readIndexOffset()
+    {
+        return true;
+    }
+
+    /**
+     * When true the SSTableReader should only read repaired SSTables from a single 'primary repair' replica
+     * and read unrepaired SSTables at the user set consistency level
+     *
+     * @return true if the SSTableReader should only read repaired SSTables on single 'repair primary' replica
+     */
+    public boolean useIncrementalRepair()
+    {
+        return true;
+    }
+
+    /**
+     * @return CompactionScanner for iterating over one or more SSTables, compacting data and purging tombstones
+     */
+    public StreamScanner openCompactionScanner(int partitionId,
+                                               List<PartitionKeyFilter> partitionKeyFilters,
+                                               @Nullable PruneColumnFilter columnFilter)
+    {
+        List<PartitionKeyFilter> filtersInRange;
+        try
+        {
+            filtersInRange = partitionKeyFiltersInRange(partitionId, partitionKeyFilters);
+        }
+        catch (NoMatchFoundException exception)
+        {
+            return EmptyStreamScanner.INSTANCE;
+        }
+        SparkRangeFilter sparkRangeFilter = sparkRangeFilter(partitionId);
+        return bridge().getCompactionScanner(cqlTable(),
+                                             partitioner(),
+                                             sstables(partitionId, sparkRangeFilter, filtersInRange),
+                                             sparkRangeFilter,
+                                             filtersInRange,
+                                             columnFilter,
+                                             timeProvider(),
+                                             readIndexOffset(),
+                                             useIncrementalRepair(),
+                                             stats());
+    }
+
+    /**
+     * @return a TimeProvider that returns the time now in seconds. User can override with their own provider
+     */
+    public TimeProvider timeProvider()
+    {
+        return bridge().timeProvider();
+    }
+
+    /**
+     * @param filters array of push down filters that
+     * @return an array of push filters that are <b>not</b> supported by this data layer
+     */
+    public Filter[] unsupportedPushDownFilters(Filter[] filters)
+    {
+        Set<String> partitionKeys = cqlTable().partitionKeys().stream()
+                                              .map(key -> StringUtils.lowerCase(key.name()))
+                                              .collect(Collectors.toSet());
+
+        List<Filter> unsupportedFilters = new ArrayList<>(filters.length);
+        for (Filter filter : filters)
+        {
+            if (filter instanceof EqualTo || filter instanceof In)
+            {
+                String columnName = StringUtils.lowerCase(filter instanceof EqualTo
+                        ? ((EqualTo) filter).attribute()
+                        : ((In) filter).attribute());
+
+                if (partitionKeys.contains(columnName))
+                {
+                    partitionKeys.remove(columnName);
+                }
+                else
+                {
+                    // Only partition keys are supported
+                    unsupportedFilters.add(filter);
+                }
+            }
+            else
+            {
+                // Push down filters other than EqualTo & In not supported yet
+                unsupportedFilters.add(filter);
+            }
+        }
+        // If the partition keys are not in the filter, we disable push down
+        return partitionKeys.size() > 0 ? filters : unsupportedFilters.toArray(new Filter[0]);
+    }
+
+    /**
+     * Override to plug in your own Stats instrumentation for recording internal events
+     *
+     * @return Stats implementation to record internal events
+     */
+    public Stats stats()
+    {
+        return Stats.DoNothingStats.INSTANCE;
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/data/FileType.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/data/FileType.java
new file mode 100644
index 0000000..e940a1b
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/data/FileType.java
@@ -0,0 +1,83 @@
+/*
+ * 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.cassandra.spark.data;
+
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.HashMap;
+import java.util.Map;
+
+import com.google.common.base.Preconditions;
+
+import org.jetbrains.annotations.Nullable;
+
+public enum FileType
+{
+    DATA("Data.db"),
+    INDEX("Index.db"),
+    FILTER("Filter.db"),
+    STATISTICS("Statistics.db"),
+    SUMMARY("Summary.db"),
+    COMPRESSION_INFO("CompressionInfo.db"),
+    TOC("TOC.txt"),
+    DIGEST("Digest.sha1"),
+    CRC("CRC.db"),
+    CRC32("Digest.crc32"),
+    COMMITLOG(".log");
+
+    private final String fileSuffix;
+
+    FileType(String fileSuffix)
+    {
+        this.fileSuffix = fileSuffix;
+    }
+
+    private static final Map<String, FileType> FILE_TYPE_HASH_MAP = new HashMap<>();
+
+    static
+    {
+        for (FileType fileType : FileType.values())
+        {
+            FILE_TYPE_HASH_MAP.put(fileType.getFileSuffix(), fileType);
+        }
+    }
+
+    public static FileType fromExtension(String extension)
+    {
+        Preconditions.checkArgument(FILE_TYPE_HASH_MAP.containsKey(extension),
+                                    "Unknown sstable file type: " + extension);
+        return FILE_TYPE_HASH_MAP.get(extension);
+    }
+
+    @Nullable
+    public static Path resolveComponentFile(FileType fileType, Path dataFilePath)
+    {
+        Path filePath = fileType == FileType.DATA ? dataFilePath : dataFilePath.resolveSibling(dataFilePath
+                .getFileName()
+                .toString()
+                .replace(FileType.DATA.getFileSuffix(), fileType.getFileSuffix()));
+        return Files.exists(filePath) ? filePath : null;
+    }
+
+    public String getFileSuffix()
+    {
+        return fileSuffix;
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/data/IncompleteSSTableException.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/data/IncompleteSSTableException.java
new file mode 100644
index 0000000..0a6663e
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/data/IncompleteSSTableException.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.cassandra.spark.data;
+
+import java.util.Arrays;
+
+import org.apache.cassandra.spark.reader.common.SSTableStreamException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Thrown when a snapshot of SSTables is incomplete so cannot be used
+ */
+@SuppressWarnings({"unused", "WeakerAccess"})
+public class IncompleteSSTableException extends SSTableStreamException
+{
+    public IncompleteSSTableException(FileType... fileTypes)
+    {
+        super(String.format("SSTable file component '%s' is required but could not be found",
+                            Arrays.toString(fileTypes)));
+    }
+
+    public static boolean isIncompleteException(@Nullable Throwable throwable)
+    {
+        return throwable != null && (throwable instanceof IncompleteSSTableException || isIncompleteException(throwable.getCause()));
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/data/ReplicationFactor.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/data/ReplicationFactor.java
new file mode 100644
index 0000000..df2cb1f
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/data/ReplicationFactor.java
@@ -0,0 +1,224 @@
+/*
+ * 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.cassandra.spark.data;
+
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+import org.apache.commons.lang.builder.EqualsBuilder;
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Replication factor object, expected format:
+ *     {
+ *         "class" : "NetworkTopologyStrategy",
+ *         "options" : {
+ *             "DC1" : 2,
+ *             "DC2" : 2
+ *         }
+ *     }
+ *     {
+ *         "class" : "SimpleStrategy",
+ *         "options" : {
+ *             "replication_factor" : 1
+ *         }
+ *     }
+ */
+public class ReplicationFactor implements Serializable
+{
+    private static final long serialVersionUID = -2017022813595983257L;
+    private static final Logger LOGGER = LoggerFactory.getLogger(ReplicationFactor.class);
+
+    public enum ReplicationStrategy
+    {
+        LocalStrategy(0),
+        SimpleStrategy(1),
+        NetworkTopologyStrategy(2);
+
+        public final int value;
+
+        ReplicationStrategy(int value)
+        {
+            this.value = value;
+        }
+
+        public static ReplicationStrategy valueOf(int value)
+        {
+            switch (value)
+            {
+                case 0:
+                    return LocalStrategy;
+                case 1:
+                    return SimpleStrategy;
+                case 2:
+                    return NetworkTopologyStrategy;
+                default:
+                    throw new IllegalStateException("Unknown ReplicationStrategy: " + value);
+            }
+        }
+
+        public static ReplicationStrategy getEnum(String value)
+        {
+            for (ReplicationStrategy v : values())
+            {
+                if (value.equalsIgnoreCase(v.name()) || value.endsWith("." + v.name()))
+                {
+                    return v;
+                }
+            }
+            throw new IllegalArgumentException();
+        }
+    }
+
+    @NotNull
+    private final ReplicationStrategy replicationStrategy;
+    @NotNull
+    private final Map<String, Integer> options;
+
+    public ReplicationFactor(@NotNull Map<String, String> options)
+    {
+        this.replicationStrategy = ReplicationFactor.ReplicationStrategy.getEnum(options.get("class"));
+        this.options = new LinkedHashMap<>(options.size());
+        for (Map.Entry<String, String> entry : options.entrySet())
+        {
+            if ("class".equals(entry.getKey()))
+            {
+                continue;
+            }
+
+            try
+            {
+                this.options.put(entry.getKey(), Integer.parseInt(entry.getValue()));
+            }
+            catch (NumberFormatException exception)
+            {
+                LOGGER.warn("Could not parse replication option: {} = {}", entry.getKey(), entry.getValue());
+            }
+        }
+    }
+
+    public ReplicationFactor(@NotNull ReplicationStrategy replicationStrategy, @NotNull Map<String, Integer> options)
+    {
+        this.replicationStrategy = replicationStrategy;
+        this.options = new LinkedHashMap<>(options.size());
+
+        if (!replicationStrategy.equals(ReplicationStrategy.LocalStrategy) && options.isEmpty())
+        {
+            throw new RuntimeException(String.format("Could not find replication info in schema map: %s.", options));
+        }
+
+        for (Map.Entry<String, Integer> entry : options.entrySet())
+        {
+            if ("class".equals(entry.getKey()))
+            {
+                continue;
+            }
+            this.options.put(entry.getKey(), entry.getValue());
+        }
+    }
+
+    public Integer getTotalReplicationFactor()
+    {
+        return options.values().stream()
+                      .mapToInt(Integer::intValue)
+                      .sum();
+    }
+
+    @NotNull
+    public Map<String, Integer> getOptions()
+    {
+        return options;
+    }
+
+    @NotNull
+    public ReplicationStrategy getReplicationStrategy()
+    {
+        return replicationStrategy;
+    }
+
+    @Override
+    public boolean equals(Object other)
+    {
+        if (other == null)
+        {
+            return false;
+        }
+        if (this == other)
+        {
+            return true;
+        }
+        if (this.getClass() != other.getClass())
+        {
+            return false;
+        }
+
+        ReplicationFactor that = (ReplicationFactor) other;
+        return new EqualsBuilder()
+               .append(this.replicationStrategy, that.replicationStrategy)
+               .append(this.options, that.options)
+               .isEquals();
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return new HashCodeBuilder()
+               .append(replicationStrategy)
+               .append(options)
+               .toHashCode();
+    }
+
+    public static class Serializer extends com.esotericsoftware.kryo.Serializer<ReplicationFactor>
+    {
+        @Override
+        public void write(Kryo kryo, Output out, ReplicationFactor replicationFactor)
+        {
+            out.writeByte(replicationFactor.replicationStrategy.value);
+            out.writeByte(replicationFactor.options.size());
+            for (Map.Entry<String, Integer> entry : replicationFactor.options.entrySet())
+            {
+                out.writeString(entry.getKey());
+                out.writeByte(entry.getValue());
+            }
+        }
+
+        @Override
+        public ReplicationFactor read(Kryo kryo, Input in, Class<ReplicationFactor> type)
+        {
+            ReplicationStrategy strategy = ReplicationStrategy.valueOf(in.readByte());
+            int numOptions = in.readByte();
+            Map<String, Integer> options = new HashMap<>(numOptions);
+            for (int option = 0; option < numOptions; option++)
+            {
+                options.put(in.readString(), (int) in.readByte());
+            }
+            return new ReplicationFactor(strategy, options);
+        }
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/data/SSTable.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/data/SSTable.java
new file mode 100644
index 0000000..4ae1198
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/data/SSTable.java
@@ -0,0 +1,102 @@
+/*
+ * 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.cassandra.spark.data;
+
+import java.io.InputStream;
+import java.io.Serializable;
+import java.util.Objects;
+
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Abstract class representing a single SSTable.
+ * Implementations must override hashCode and equals methods.
+ */
+public abstract class SSTable implements Serializable
+{
+    public static final long serialVersionUID = 42L;
+
+    public SSTable()
+    {
+    }
+
+    @Nullable
+    protected abstract InputStream openInputStream(FileType fileType);
+
+    @Nullable
+    public InputStream openCompressionStream()
+    {
+        return openInputStream(FileType.COMPRESSION_INFO);
+    }
+
+    @Nullable
+    public InputStream openStatsStream()
+    {
+        return openInputStream(FileType.STATISTICS);
+    }
+
+    @Nullable
+    public InputStream openSummaryStream()
+    {
+        return openInputStream(FileType.SUMMARY);
+    }
+
+    @Nullable
+    public InputStream openPrimaryIndexStream()
+    {
+        return openInputStream(FileType.INDEX);
+    }
+
+    @Nullable
+    public InputStream openFilterStream()
+    {
+        return openInputStream(FileType.FILTER);
+    }
+
+    @NotNull
+    public InputStream openDataStream()
+    {
+        return Objects.requireNonNull(openInputStream(FileType.DATA), "Data.db SSTable file component must exist");
+    }
+
+    public abstract boolean isMissing(FileType fileType);
+
+    public void verify() throws IncompleteSSTableException
+    {
+        // Need Data.db file
+        if (isMissing(FileType.DATA))
+        {
+            throw new IncompleteSSTableException(FileType.DATA);
+        }
+        // Need Statistics.db file to open SerializationHeader
+        if (isMissing(FileType.STATISTICS))
+        {
+            throw new IncompleteSSTableException(FileType.STATISTICS);
+        }
+        // Need Summary.db or Index.db to read first/last partition key
+        if (isMissing(FileType.SUMMARY) && isMissing(FileType.INDEX))
+        {
+            throw new IncompleteSSTableException(FileType.SUMMARY, FileType.INDEX);
+        }
+    }
+
+    public abstract String getDataFileName();
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/data/SSTablesSupplier.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/data/SSTablesSupplier.java
new file mode 100644
index 0000000..0f4745d
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/data/SSTablesSupplier.java
@@ -0,0 +1,46 @@
+/*
+ * 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.cassandra.spark.data;
+
+import java.io.IOException;
+import java.util.Set;
+
+import org.apache.cassandra.spark.reader.SparkSSTableReader;
+
+public abstract class SSTablesSupplier
+{
+    /**
+     * Interface that opens a given SSTable and returns an SparkSSTableReader
+     *
+     * @param <T> SparkSSTableReader
+     */
+    public interface ReaderOpener<T extends SparkSSTableReader>
+    {
+        T openReader(SSTable ssTable, boolean isRepairPrimary) throws IOException;
+    }
+
+    /**
+     * Open an SparkSSTableReader for each SSTable
+     *
+     * @param readerOpener open SparkSSTableReader for a given SSTable
+     * @return set of open SparkSSTableReaders to pass into CompactionIterator
+     */
+    public abstract <T extends SparkSSTableReader> Set<T> openAll(ReaderOpener<T> readerOpener);
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/data/partitioner/CassandraInstance.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/data/partitioner/CassandraInstance.java
new file mode 100644
index 0000000..b868f54
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/data/partitioner/CassandraInstance.java
@@ -0,0 +1,121 @@
+/*
+ * 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.cassandra.spark.data.partitioner;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+import org.apache.commons.lang.builder.EqualsBuilder;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+import org.apache.cassandra.spark.cdc.CommitLog;
+
+@SuppressWarnings("WeakerAccess")
+public class CassandraInstance implements Serializable
+{
+    private static final long serialVersionUID = 6767636627576239773L;
+    private final String token;
+    private final String node;
+    private final String dataCenter;
+
+    public CassandraInstance(String token, String node, String dataCenter)
+    {
+        this.token = token;
+        this.node = node;
+        this.dataCenter = dataCenter;
+    }
+
+    public String token()
+    {
+        return token;
+    }
+
+    public String nodeName()
+    {
+        return node;
+    }
+
+    public String dataCenter()
+    {
+        return dataCenter;
+    }
+
+    @SuppressWarnings("unused")
+    public CommitLog.Marker zeroMarker()
+    {
+        return markerAt(0, 0);
+    }
+
+    public CommitLog.Marker markerAt(long section, int position)
+    {
+        return new CommitLog.Marker(this, section, position);
+    }
+
+    @Override
+    public boolean equals(Object other)
+    {
+        if (this == other)
+        {
+            return true;
+        }
+        if (other == null || this.getClass() != other.getClass())
+        {
+            return false;
+        }
+
+        CassandraInstance that = (CassandraInstance) other;
+        return new EqualsBuilder()
+               .append(this.token, that.token)
+               .append(this.node, that.node)
+               .append(this.dataCenter, that.dataCenter)
+               .isEquals();
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Objects.hash(token, node, dataCenter);
+    }
+
+    @Override
+    public String toString()
+    {
+        return String.format("{\"token\"=\"%s\", \"node\"=\"%s\", \"dc\"=\"%s\"}", token, node, dataCenter);
+    }
+
+    public static class Serializer extends com.esotericsoftware.kryo.Serializer<CassandraInstance>
+    {
+        @Override
+        public CassandraInstance read(Kryo kryo, Input in, Class type)
+        {
+            return new CassandraInstance(in.readString(), in.readString(), in.readString());
+        }
+
+        @Override
+        public void write(Kryo kryo, Output out, CassandraInstance instance)
+        {
+            out.writeString(instance.token());
+            out.writeString(instance.nodeName());
+            out.writeString(instance.dataCenter());
+        }
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/data/partitioner/MurmurHash.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/data/partitioner/MurmurHash.java
new file mode 100644
index 0000000..ae3722e
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/data/partitioner/MurmurHash.java
@@ -0,0 +1,159 @@
+/*
+ * 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.cassandra.spark.data.partitioner;
+
+import java.nio.ByteBuffer;
+
+/**
+ * This is a very fast, non-cryptographic hash suitable for general hash-based lookup.
+ * See http://murmurhash.googlepages.com/ for more details.
+ *
+ * hash32() and hash64() are MurmurHash 2.0.
+ * hash3_x64_128() is MurmurHash 3.0.
+ *
+ * The C version of MurmurHash 2.0 found at that site was ported to Java by Andrzej Bialecki.
+ */
+public final class MurmurHash
+{
+    private MurmurHash()
+    {
+        throw new IllegalStateException(getClass() + " is static utility class and shall not be instantiated");
+    }
+
+    private static long block(ByteBuffer key, int offset, int index)
+    {
+        int blockOffset = offset + (index << 3);
+        return ((long) key.get(blockOffset + 0) & 0xFF)
+            + (((long) key.get(blockOffset + 1) & 0xFF) <<  8)
+            + (((long) key.get(blockOffset + 2) & 0xFF) << 16)
+            + (((long) key.get(blockOffset + 3) & 0xFF) << 24)
+            + (((long) key.get(blockOffset + 4) & 0xFF) << 32)
+            + (((long) key.get(blockOffset + 5) & 0xFF) << 40)
+            + (((long) key.get(blockOffset + 6) & 0xFF) << 48)
+            + (((long) key.get(blockOffset + 7) & 0xFF) << 56);
+    }
+
+    private static long rotate(long value, int shift)
+    {
+        return (value << shift) | (value >>> (64 - shift));
+    }
+
+    private static long mix(long value)
+    {
+        value ^= value >>> 33;
+        value *= 0xFF51AFD7ED558CCDL;
+        value ^= value >>> 33;
+        value *= 0xC4CEB9FE1A85EC53L;
+        value ^= value >>> 33;
+
+        return value;
+    }
+
+    public static long[] hash(ByteBuffer key, int offset, int length, long seed)
+    {
+        int nblocks = length >> 4;  // Process as 128-bit blocks
+
+        long h1 = seed;
+        long h2 = seed;
+
+        long c1 = 0x87C37B91114253D5L;
+        long c2 = 0x4CF5AD432745937FL;
+
+        // Body
+
+        for (int block = 0; block < nblocks; block++)
+        {
+            long k1 = block(key, offset, block * 2 + 0);
+            long k2 = block(key, offset, block * 2 + 1);
+
+            k1 *= c1;
+            k1 = rotate(k1, 31);
+            k1 *= c2;
+            h1 ^= k1;
+
+            h1 = rotate(h1, 27);
+            h1 += h2;
+            h1 = h1 * 5 + 0x52DCE729;
+
+            k2 *= c2;
+            k2 = rotate(k2, 33);
+            k2 *= c1;
+            h2 ^= k2;
+
+            h2 = rotate(h2, 31);
+            h2 += h1;
+            h2 = h2 * 5 + 0x38495AB5;
+        }
+
+        // Tail
+
+        // Advance offset to the unprocessed tail of the data
+        offset += nblocks * 16;
+
+        long k1 = 0;
+        long k2 = 0;
+
+        switch (length & 15)
+        {
+            case 15: k2 ^= ((long) key.get(offset + 14)) << 48;
+            case 14: k2 ^= ((long) key.get(offset + 13)) << 40;
+            case 13: k2 ^= ((long) key.get(offset + 12)) << 32;
+            case 12: k2 ^= ((long) key.get(offset + 11)) << 24;
+            case 11: k2 ^= ((long) key.get(offset + 10)) << 16;
+            case 10: k2 ^= ((long) key.get(offset +  9)) <<  8;
+            case  9: k2 ^= ((long) key.get(offset +  8));
+                     k2 *= c2;
+                     k2  = rotate(k2, 33);
+                     k2 *= c1;
+                     h2 ^= k2;
+
+            case  8: k1 ^= ((long) key.get(offset + 7)) << 56;
+            case  7: k1 ^= ((long) key.get(offset + 6)) << 48;
+            case  6: k1 ^= ((long) key.get(offset + 5)) << 40;
+            case  5: k1 ^= ((long) key.get(offset + 4)) << 32;
+            case  4: k1 ^= ((long) key.get(offset + 3)) << 24;
+            case  3: k1 ^= ((long) key.get(offset + 2)) << 16;
+            case  2: k1 ^= ((long) key.get(offset + 1))  << 8;
+            case  1: k1 ^= ((long) key.get(offset));
+                     k1 *= c1;
+                     k1  = rotate(k1, 31);
+                     k1 *= c2;
+                     h1 ^= k1;
+
+            default:  // Do nothing
+        }
+
+        // Finalization
+
+        h1 ^= length;
+        h2 ^= length;
+
+        h1 += h2;
+        h2 += h1;
+
+        h1 = mix(h1);
+        h2 = mix(h2);
+
+        h1 += h2;
+        h2 += h1;
+
+        return new long[]{h1, h2};
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/data/partitioner/Partitioner.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/data/partitioner/Partitioner.java
new file mode 100644
index 0000000..f36bc21
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/data/partitioner/Partitioner.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.cassandra.spark.data.partitioner;
+
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.security.MessageDigest;
+import java.util.function.Function;
+
+/**
+ * Encapsulates partitioner ranges and the ability to compute the hash for a given key. Supports:
+ *
+ * <ul>
+ *     <li>RandomPartitioner
+ *     <li>Murmur3Partitioner
+ * </ul>
+ */
+public enum Partitioner
+{
+    RandomPartitioner(BigInteger.ZERO, BigInteger.valueOf(2).pow(127).subtract(BigInteger.ONE), key -> {
+        // Random partitioner hash using MD5
+        MessageDigest md;
+        try
+        {
+            md = MessageDigest.getInstance("MD5");
+        }
+        catch (Exception exception)
+        {
+            throw new RuntimeException(exception);
+        }
+        md.reset();
+        md.update(key);
+        return new BigInteger(md.digest()).abs();
+    }),
+    Murmur3Partitioner(BigInteger.valueOf(2).pow(63).negate(), BigInteger.valueOf(2).pow(63).subtract(BigInteger.ONE), key -> {
+        // Otherwise Murmur3 hash
+        long value = MurmurHash.hash(key, key.position(), key.remaining(), 0)[0];
+        value = value == Long.MIN_VALUE ? Long.MAX_VALUE : value;  // Normalize
+        return BigInteger.valueOf(value);
+    });
+
+    /**
+     * Returns the {@link Partitioner} type based on the class name. It matches the fully qualified class name, or
+     * class name.
+     *
+     * @param partitionerClass the class name or the fully qualified class name
+     * @return the {@link Partitioner} type based on the class name
+     */
+    public static Partitioner from(String partitionerClass)
+    {
+        switch (partitionerClass)
+        {
+            case "org.apache.cassandra.dht.Murmur3Partitioner":
+            case "Murmur3Partitioner":
+                return Partitioner.Murmur3Partitioner;
+
+            case "org.apache.cassandra.dht.RandomPartitioner":
+            case "RandomPartitioner":
+                return Partitioner.RandomPartitioner;
+
+            default:
+                throw new UnsupportedOperationException("Unexpected partitioner: " + partitionerClass);
+        }
+    }
+
+    private final BigInteger minToken;
+    private final BigInteger maxToken;
+    private final Function<ByteBuffer, BigInteger> hash;
+
+    Partitioner(BigInteger minToken, BigInteger maxToken, Function<ByteBuffer, BigInteger> hash)
+    {
+        this.minToken = minToken;
+        this.maxToken = maxToken;
+        this.hash = hash;
+    }
+
+    public BigInteger minToken()
+    {
+        return minToken;
+    }
+
+    public BigInteger maxToken()
+    {
+        return maxToken;
+    }
+
+    public BigInteger hash(ByteBuffer key)
+    {
+        return hash.apply(key);
+    }
+
+    @Override
+    public String toString()
+    {
+        return "org.apache.cassandra.dht." + super.toString();
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/exceptions/TransportFailureException.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/exceptions/TransportFailureException.java
new file mode 100644
index 0000000..10d07e7
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/exceptions/TransportFailureException.java
@@ -0,0 +1,97 @@
+/*
+ * 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.cassandra.spark.exceptions;
+
+import org.jetbrains.annotations.Nullable;
+
+public class TransportFailureException extends RuntimeException
+{
+    public static final int NOT_FOUND_404 = 404;
+
+    @Nullable
+    protected final transient Integer statusCode;
+
+    public static TransportFailureException of(String message, Throwable cause)
+    {
+        return new TransportFailureException(null, message, cause);
+    }
+
+    public static TransportFailureException nonretryable(int statusCode)
+    {
+        return new Nonretryable(statusCode);
+    }
+
+    public static TransportFailureException retryExhausted(String message, Throwable cause)
+    {
+        return new RetryExhausted(message, cause);
+    }
+
+    public static TransportFailureException unexpectedResponseType(int statusCode)
+    {
+        return new UnexpectedResponseType(statusCode);
+    }
+
+    public boolean isNotFound()
+    {
+        return statusCode != null && statusCode == NOT_FOUND_404;
+    }
+
+    public TransportFailureException(int statusCode, String message)
+    {
+        super(message);
+        this.statusCode = statusCode;
+    }
+
+    public TransportFailureException(@Nullable Integer statusCode, String message, Throwable cause)
+    {
+        super(message, cause);
+        this.statusCode = statusCode;
+    }
+
+    public TransportFailureException(@Nullable Integer statusCode, Throwable cause)
+    {
+        super(cause);
+        this.statusCode = statusCode;
+    }
+
+    public static class Nonretryable extends TransportFailureException
+    {
+        public Nonretryable(int statusCode)
+        {
+            super(statusCode, "Non-retryable status code: " + statusCode);
+        }
+    }
+
+    public static class UnexpectedResponseType extends TransportFailureException
+    {
+        public UnexpectedResponseType(int statusCode)
+        {
+            super(statusCode, "Unexpected http response type: " + statusCode);
+        }
+    }
+
+    public static class RetryExhausted extends TransportFailureException
+    {
+        public RetryExhausted(String message, Throwable cause)
+        {
+            super(null, message, cause);
+        }
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/reader/EmptyStreamScanner.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/reader/EmptyStreamScanner.java
new file mode 100644
index 0000000..b4c8e0e
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/reader/EmptyStreamScanner.java
@@ -0,0 +1,47 @@
+/*
+ * 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.cassandra.spark.reader;
+
+public class EmptyStreamScanner implements StreamScanner
+{
+    public static final EmptyStreamScanner INSTANCE = new EmptyStreamScanner();
+
+    @Override
+    public Rid rid()
+    {
+        return null;
+    }
+
+    @Override
+    public boolean hasNext()
+    {
+        return false;
+    }
+
+    @Override
+    public void advanceToNextColumn()
+    {
+    }
+
+    @Override
+    public void close()
+    {
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/reader/Rid.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/reader/Rid.java
new file mode 100644
index 0000000..ff1d07c
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/reader/Rid.java
@@ -0,0 +1,238 @@
+/*
+ * 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.cassandra.spark.reader;
+
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+
+import org.apache.cassandra.spark.sparksql.RangeTombstoneMarker;
+import org.apache.cassandra.spark.utils.ByteBufferUtils;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Rid - Row Identifier - contains the partition key, clustering keys and column name that uniquely identifies a row and column of data in Cassandra
+ */
+public class Rid
+{
+    private ByteBuffer partitionKey;
+    private ByteBuffer columnName;
+    private ByteBuffer value;
+    private long timestamp;
+    private BigInteger token;
+    @VisibleForTesting
+    boolean isNewPartition = false;
+    private boolean isRowDeletion = false;
+    private boolean isPartitionDeletion = false;
+    private boolean isUpdate = false;
+
+    // Optional field; memorizes tombstoned elements/cells in a complex data; only used in CDC
+    @Nullable
+    private List<ByteBuffer> tombstonedCellsInComplex = null;
+
+    // Optional field; memorizes the range tombstone markers with in the same partition; only used in CDC
+    private List<RangeTombstoneMarker> rangeTombstoneMarkers = null;
+    private boolean shouldConsumeRangeTombstoneMarkers = false;
+
+    // Partition Key Value
+
+    public void setPartitionKeyCopy(ByteBuffer partitionKeyBytes, BigInteger token)
+    {
+        this.partitionKey = partitionKeyBytes;
+        this.token = token;
+        this.columnName = null;
+        this.value = null;
+        this.isNewPartition = true;
+        this.timestamp = 0L;
+    }
+
+    public boolean isNewPartition()
+    {
+        if (isNewPartition)
+        {
+            isNewPartition = false;
+            return true;
+        }
+        return false;
+    }
+
+    public boolean isPartitionDeletion()
+    {
+        return isPartitionDeletion;
+    }
+
+    public void setPartitionDeletion(boolean isPartitionDeletion)
+    {
+        this.isPartitionDeletion = isPartitionDeletion;
+    }
+
+    public boolean isUpdate()
+    {
+        return isUpdate;
+    }
+
+    public void setIsUpdate(boolean isUpdate)
+    {
+        this.isUpdate = isUpdate;
+    }
+
+    public boolean isRowDeletion()
+    {
+        return isRowDeletion;
+    }
+
+    public void setRowDeletion(boolean isRowDeletion)
+    {
+        this.isRowDeletion = isRowDeletion;
+    }
+
+    public ByteBuffer getPartitionKey()
+    {
+        return partitionKey;
+    }
+
+    public BigInteger getToken()
+    {
+        return token;
+    }
+
+    // Column Name (contains concatenated clustering keys and column name)
+
+    public void setColumnNameCopy(ByteBuffer columnBytes)
+    {
+        this.columnName = columnBytes;
+    }
+
+    public ByteBuffer getColumnName()
+    {
+        return columnName;
+    }
+
+    // Value of Cell
+
+    public ByteBuffer getValue()
+    {
+        return value;
+    }
+
+    public void setValueCopy(ByteBuffer value)
+    {
+        this.value = value;
+    }
+
+    // Timestamp
+
+    public void setTimestamp(long timestamp)
+    {
+        this.timestamp = timestamp;
+    }
+
+    public long getTimestamp()
+    {
+        return timestamp;
+    }
+
+    // CDC: Handle element deletion in complex, adds the serialized cellpath to the tombstone
+    public void addCellTombstoneInComplex(ByteBuffer key)
+    {
+        if (tombstonedCellsInComplex == null)
+        {
+            tombstonedCellsInComplex = new ArrayList<>();
+        }
+        tombstonedCellsInComplex.add(key);
+    }
+
+    public boolean hasCellTombstoneInComplex()
+    {
+        return tombstonedCellsInComplex != null && !tombstonedCellsInComplex.isEmpty();
+    }
+
+    public List<ByteBuffer> getCellTombstonesInComplex()
+    {
+        return tombstonedCellsInComplex;
+    }
+
+    public void resetCellTombstonesInComplex()
+    {
+        tombstonedCellsInComplex = null;
+    }
+
+    public void addRangeTombstoneMarker(RangeTombstoneMarker marker)
+    {
+        if (rangeTombstoneMarkers == null)
+        {
+            rangeTombstoneMarkers = new ArrayList<>();
+        }
+
+        // Ensure the marker list is valid
+        if (rangeTombstoneMarkers.isEmpty())
+        {
+            Preconditions.checkArgument(!marker.isBoundary() && marker.isOpen(false),
+                                        "The first marker should be an open bound");
+            rangeTombstoneMarkers.add(marker);
+        }
+        else
+        {
+            RangeTombstoneMarker lastMarker = rangeTombstoneMarkers.get(rangeTombstoneMarkers.size() - 1);
+            Preconditions.checkArgument((lastMarker.isOpen(false) && marker.isClose(false))
+                                     || (lastMarker.isClose(false) && marker.isOpen(false)),
+                                        "Current marker should close or open a new range");
+            rangeTombstoneMarkers.add(marker);
+        }
+    }
+
+    public boolean hasRangeTombstoneMarkers()
+    {
+        return rangeTombstoneMarkers != null && !rangeTombstoneMarkers.isEmpty();
+    }
+
+    public void setShouldConsumeRangeTombstoneMarkers(boolean shouldConsumeRangeTombstoneMarkers)
+    {
+        this.shouldConsumeRangeTombstoneMarkers = shouldConsumeRangeTombstoneMarkers;
+    }
+
+    public boolean shouldConsumeRangeTombstoneMarkers()
+    {
+        return shouldConsumeRangeTombstoneMarkers;
+    }
+
+    public List<RangeTombstoneMarker> getRangeTombstoneMarkers()
+    {
+        return rangeTombstoneMarkers;
+    }
+
+    public void resetRangeTombstoneMarkers()
+    {
+        rangeTombstoneMarkers = null;
+        shouldConsumeRangeTombstoneMarkers = false;
+    }
+
+    @Override
+    public String toString()
+    {
+        return ByteBufferUtils.toHexString(getPartitionKey()) + ":"
+             + ByteBufferUtils.toHexString(getColumnName()) + ":"
+             + ByteBufferUtils.toHexString(getValue());
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/reader/SparkSSTableReader.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/reader/SparkSSTableReader.java
new file mode 100644
index 0000000..3a53fb6
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/reader/SparkSSTableReader.java
@@ -0,0 +1,54 @@
+/*
+ * 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.cassandra.spark.reader;
+
+import java.math.BigInteger;
+
+import org.apache.cassandra.bridge.TokenRange;
+
+public interface SparkSSTableReader
+{
+    BigInteger firstToken();
+
+    BigInteger lastToken();
+
+    default TokenRange range()
+    {
+        // It is possible for the range boundaries to be inverted in unsorted SSTables produced by
+        // the writer of Cassandra version 3.0, at least this is what happens when running unit tests
+        return firstToken().compareTo(lastToken()) <= 0 ? TokenRange.closed(firstToken(), lastToken())
+                                                        : TokenRange.closed(lastToken(), firstToken());
+    }
+
+    /**
+     * @return true if this SSTable should not be read as part of this Spark partition
+     */
+    boolean ignore();
+
+    /**
+     * @param reader SSTable reader
+     * @param range  token range
+     * @return true if SSTable reader overlaps with a given token range
+     */
+    static boolean overlaps(SparkSSTableReader reader, TokenRange range)
+    {
+        return range.isConnected(reader.range());
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/reader/StreamScanner.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/reader/StreamScanner.java
new file mode 100644
index 0000000..2f58f19
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/reader/StreamScanner.java
@@ -0,0 +1,79 @@
+/*
+ * 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.cassandra.spark.reader;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/**
+ * A rid is just a pair of ids that uniquely identifies the row and the column of a data entity.
+ * Reading a Cassandra SSTable pivots the data in a way that projects all columns against the rows
+ * they belong to:
+ * <p>
+ * Cassandra:
+ *     r1 | c1, c2, c3
+ *     r2 | c4
+ *     r3 | c5, c6, c7, c8
+ * <p>
+ * Pivoted:
+ *     r1 | c1
+ *     r1 | c2
+ *     r1 | c3
+ *     r2 | c4
+ *     r3 | c5
+ *     r3 | c6
+ *     r3 | c7
+ *     r3 | c8
+ * <p>
+ * During a loading operation we will extract up to a few trillion items out of SSTables, so it is of
+ * high importance to reuse objects - the caller to the scanner creates a rid using the
+ * callers implementation of those interfaces; the scanner then calls set**Copy() to provide the data
+ * at which point the implementation should make a copy of the provided bytes.
+ * <p>
+ * Upon return from the next() call the current values of the scanner can be obtained by calling
+ * the methods in Rid, getPartitionKey(), getColumnName(), getValue().
+ */
+@SuppressWarnings("unused")
+public interface StreamScanner extends Closeable
+{
+    /**
+     * Expose the data/rid to be consumed.
+     * Implementation note: rid should always be updated to the current partition if hasNext returns true.
+     *
+     * @return rid
+     */
+    Rid rid();
+
+    /**
+     * Indicate if there are more data/rid avaiable
+     *
+     * @return true when the rid is available to be consumed;
+     *         otherwise, return false to indicate the scanner has exhausted
+     * @throws IOException
+     */
+    boolean hasNext() throws IOException;
+
+    /**
+     * Consume the data from the next column and store in rid
+     *
+     * @throws IOException
+     */
+    void advanceToNextColumn() throws IOException;
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/reader/common/AbstractCompressionMetadata.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/reader/common/AbstractCompressionMetadata.java
new file mode 100644
index 0000000..985b827
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/reader/common/AbstractCompressionMetadata.java
@@ -0,0 +1,101 @@
+/*
+ * 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.cassandra.spark.reader.common;
+
+import java.io.EOFException;
+import java.io.IOException;
+
+public abstract class AbstractCompressionMetadata
+{
+    // From 3.0, CRC check chance is part of table schema not compression metadata
+    protected static final double CRC_CHECK_CHANCE = 1.0;
+
+    private final long dataLength;
+    private final BigLongArray chunkOffsets;
+
+    protected AbstractCompressionMetadata(long dataLength, BigLongArray chunkOffsets)
+    {
+        this.dataLength = dataLength;
+        this.chunkOffsets = chunkOffsets;
+    }
+
+    protected abstract int chunkLength();
+
+    protected abstract double crcCheckChance();
+
+    public long getDataLength()
+    {
+        return dataLength;
+    }
+
+    private Chunk chunkAtIndex(int index)
+    {
+        long chunkOffset = chunkOffsets.get(index);
+        long nextChunkOffset = (index + 1 == chunkOffsets.size) ? -1 : chunkOffsets.get(index + 1);
+
+        // "4" bytes reserved for checksum
+        return new Chunk(chunkOffset, (nextChunkOffset == -1) ? -1 : (int) (nextChunkOffset - chunkOffset - 4));
+    }
+
+    /**
+     * Get a chunk of compressed data (offset, length) corresponding to given position
+     *
+     * @param position Position in the file
+     * @return chunk offset and length
+     * @throws IOException on any I/O error
+     */
+    public Chunk chunkAtPosition(long position) throws IOException
+    {
+        // Position of the chunk
+        int index = (int) (position / chunkLength());
+
+        if (index >= chunkOffsets.size)
+        {
+            throw new EOFException();
+        }
+
+        return chunkAtIndex(index);
+    }
+
+    /**
+     * Holds offset and length of the file chunk
+     */
+    public static class Chunk
+    {
+        public final long offset;
+        public int length;  // CHECKSTYLE IGNORE: Public mutable field
+
+        Chunk(long offset, int length)
+        {
+            this.offset = offset;
+            this.length = length;
+        }
+
+        public void setLength(int length)
+        {
+            this.length = length;
+        }
+
+        public String toString()
+        {
+            return String.format("Chunk<offset: %d, length: %d>", offset, length);
+        }
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/reader/common/BigLongArray.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/reader/common/BigLongArray.java
new file mode 100644
index 0000000..45ad994
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/reader/common/BigLongArray.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.cassandra.spark.reader.common;
+
+/**
+ * A GC friendly long[].
+ * Allocating large arrays (that are not short-lived) generate fragmentation
+ * in old-gen space. This breaks such large long array into fixed size pages
+ * to avoid that problem.
+ */
+public class BigLongArray
+{
+    static final int DEFAULT_PAGE_SIZE = 4096;
+
+    private final long[][] pages;
+    public final int size;
+    private final int pageSize;
+
+    public BigLongArray(int size)
+    {
+        if (size < 0)
+        {
+            throw new IndexOutOfBoundsException(String.format("BigLongArray size cannot be less than 0: %d)", size));
+        }
+        this.size = size;
+        this.pageSize = DEFAULT_PAGE_SIZE;
+
+        int lastPageSize = size % pageSize;
+        int fullPageCount = size / pageSize;
+        int pageCount = fullPageCount + (lastPageSize == 0 ? 0 : 1);
+        pages = new long[pageCount][];
+
+        for (int page = 0; page < fullPageCount; ++page)
+        {
+            pages[page] = new long[pageSize];
+        }
+
+        if (lastPageSize != 0)
+        {
+            pages[pages.length - 1] = new long[lastPageSize];
+        }
+    }
+
+    public void set(int index, long value)
+    {
+        checkIdx(index);
+        int page = index / pageSize;
+        int pageIdx = index % pageSize;
+        pages[page][pageIdx] = value;
+    }
+
+    public long get(int index)
+    {
+        checkIdx(index);
+        int page = index / pageSize;
+        int pageIdx = index % pageSize;
+        return pages[page][pageIdx];
+    }
+
+    private void checkIdx(int index)
+    {
+        if (index < 0 || size < index)
+        {
+            throw new IndexOutOfBoundsException(String.format("%d is not within [0, %d)", index, size));
+        }
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/reader/common/ChunkCorruptException.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/reader/common/ChunkCorruptException.java
new file mode 100644
index 0000000..31b1517
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/reader/common/ChunkCorruptException.java
@@ -0,0 +1,30 @@
+/*
+ * 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.cassandra.spark.reader.common;
+
+import java.io.IOException;
+
+public class ChunkCorruptException extends IOException
+{
+    public ChunkCorruptException(String message)
+    {
+        super(message);
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/reader/common/RawInputStream.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/reader/common/RawInputStream.java
new file mode 100644
index 0000000..2e57015
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/reader/common/RawInputStream.java
@@ -0,0 +1,251 @@
+/*
+ * 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.cassandra.spark.reader.common;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.cassandra.spark.stats.Stats;
+import org.apache.cassandra.spark.utils.ByteBufferUtils;
+import org.jetbrains.annotations.NotNull;
+
+public class RawInputStream extends InputStream
+{
+    protected final DataInputStream source;
+
+    protected final byte[] buffer;
+
+    // `bufferOffset` is the offset of the beginning of the buffer
+    protected long bufferOffset;
+    // `current` is the current position in source
+    protected long current = 0;
+    // `validBufferBytes` is the number of bytes in the buffer that are actually valid;
+    // this will be LESS than buffer capacity if buffer is not full!
+    protected int validBufferBytes = 0;
+    private boolean endOfStream = false;
+    protected final Stats stats;
+
+    public RawInputStream(DataInputStream source, byte[] buffer, Stats stats)
+    {
+        this.source = source;
+        this.buffer = buffer;
+        this.stats = stats;
+        this.stats.openedDataInputStream();
+    }
+
+    public boolean isEOF()
+    {
+        return endOfStream && finishedReadingBuffer();
+    }
+
+    private int bufferCursor()
+    {
+        return (int) (current - bufferOffset);
+    }
+
+    private boolean bufferInit = false;
+
+    protected boolean finishedReadingBuffer()
+    {
+        return current >= bufferOffset + validBufferBytes;
+    }
+
+    protected void maybeReBuffer() throws IOException
+    {
+        if (finishedReadingBuffer() || validBufferBytes < 0)
+        {
+            reBuffer();
+        }
+    }
+
+    protected void reBuffer() throws IOException
+    {
+        if (endOfStream)
+        {
+            throw new RuntimeException("Shouldn't be reading from a known EOF stream");
+        }
+
+        if (bufferInit)
+        {
+            bufferOffset += buffer.length;
+        }
+        else
+        {
+            bufferInit = true;
+        }
+
+        validBufferBytes = ByteBufferUtils.readFully(source, buffer, buffer.length);
+        stats.readBytes(validBufferBytes);
+
+        if (validBufferBytes < buffer.length)
+        {
+            endOfStream = true;
+        }
+    }
+
+    /**
+     * `current` tracks the current position in the source, this isn't necessarily total bytes read
+     * as skipping at the base InputStream might seek to the new offset without reading the bytes
+     *
+     * @return the current position in the source
+     */
+    public long position()
+    {
+        return current;
+    }
+
+    /**
+     * Perform standard in-memory skip if n is less than or equal to the number of bytes buffered in memory
+     *
+     * @param count the number of bytes to be skipped
+     * @return number of bytes skipped or -1 if not skipped
+     * @throws IOException IOException
+     */
+    protected long maybeStandardSkip(long count) throws IOException
+    {
+        if (count <= 0)
+        {
+            return 0;
+        }
+        else if (count <= remainingBytes())
+        {
+            // We've already buffered more than n bytes, so do a standard in-memory skip
+            return standardSkip(count);
+        }
+        else
+        {
+            return -1;
+        }
+    }
+
+    /**
+     * Skip any bytes already buffered in the 'buffer' array
+     *
+     * @return bytes actually skipped
+     * @throws IOException IOException
+     */
+    protected long skipBuffered() throws IOException
+    {
+        return standardSkip(remainingBytes());
+    }
+
+    public long standardSkip(long count) throws IOException
+    {
+        long actual = super.skip(count);
+        stats.skippedBytes(actual);
+        return actual;
+    }
+
+    @Override
+    public long skip(long count) throws IOException
+    {
+        long skipped = maybeStandardSkip(count);
+        if (skipped >= 0)
+        {
+            return skipped;
+        }
+        long remaining = count - skipBuffered();
+
+        // Skip remaining bytes at source
+        skipped = source.skip(remaining);
+        if (skipped > 0)
+        {
+            remaining -= skipped;
+
+            // Update current position marker to account for skipped bytes.
+            // Reset buffer so we rebuffer on next read.
+            current += skipped;
+            bufferOffset = current;
+            validBufferBytes = -1;
+            bufferInit = false;
+        }
+
+        long total = count - remaining;
+        stats.skippedBytes(total);
+        return total;
+    }
+
+    @Override
+    public int read() throws IOException
+    {
+        if (buffer == null)
+        {
+            throw new IOException();
+        }
+
+        if (isEOF())
+        {
+            return -1;
+        }
+
+        maybeReBuffer();
+
+        assert bufferOffset <= current && current < bufferOffset + validBufferBytes;
+
+        return ((int) buffer[(int) (current++ - bufferOffset)]) & 0xFF;
+    }
+
+    // -1 will be returned if there is nothing to read; higher-level methods like readInt
+    // or readFully (from RandomAccessFile) will throw EOFException but this should not
+    @Override
+    public int read(@NotNull byte[] buff, int offset, int length) throws IOException
+    {
+        if (buffer == null)
+        {
+            throw new IOException();
+        }
+
+        if (length == 0)
+        {
+            return 0;
+        }
+
+        if (isEOF())
+        {
+            return -1;
+        }
+
+        maybeReBuffer();
+
+        assert bufferOffset <= current && current < bufferOffset + validBufferBytes
+            : String.format("Current offset %d, buffer offset %d, buffer limit %d",
+                            current, bufferOffset, validBufferBytes);
+
+        int toCopy = Math.min(length, remainingBytes());
+
+        System.arraycopy(buffer, bufferCursor(), buff, offset, toCopy);
+        current += toCopy;
+
+        return toCopy;
+    }
+
+    protected int remainingBytes()
+    {
+        return validBufferBytes - bufferCursor();
+    }
+
+    @Override
+    public void close() throws IOException
+    {
+        source.close();
+        stats.closedDataInputStream();
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/reader/common/SSTableStreamException.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/reader/common/SSTableStreamException.java
new file mode 100644
index 0000000..fc013d0
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/reader/common/SSTableStreamException.java
@@ -0,0 +1,60 @@
+/*
+ * 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.cassandra.spark.reader.common;
+
+import java.io.IOException;
+
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * An unchecked wrapper around IOException
+ */
+public class SSTableStreamException extends RuntimeException
+{
+    public SSTableStreamException(String message)
+    {
+        this(new IOException(message));
+    }
+
+    public SSTableStreamException(IOException exception)
+    {
+        super(exception);
+    }
+
+    public IOException getIOException()
+    {
+        return (IOException) getCause();
+    }
+
+    @Nullable
+    public static IOException getIOException(@Nullable Throwable throwable)
+    {
+        if (throwable == null)
+        {
+            return null;
+        }
+        if (throwable instanceof SSTableStreamException)
+        {
+            return ((SSTableStreamException) throwable).getIOException();
+        }
+
+        return getIOException(throwable.getCause());
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/Cell.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/Cell.java
new file mode 100644
index 0000000..03493db
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/Cell.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.cassandra.spark.sparksql;
+
+class Cell
+{
+    final Object[] values;
+    final int position;
+    final boolean isNewRow;
+    final boolean isUpdate;
+    final long timestamp;
+
+    Cell(Object[] values, int position, boolean isNewRow, boolean isUpdate, long timestamp)
+    {
+        this.values = values;
+        this.position = position;
+        this.isNewRow = isNewRow;
+        this.isUpdate = isUpdate;
+        this.timestamp = timestamp;
+    }
+
+    boolean isTombstone()
+    {
+        return false;
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/CellTombstonesInComplexDecorator.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/CellTombstonesInComplexDecorator.java
new file mode 100644
index 0000000..a7a20de
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/CellTombstonesInComplexDecorator.java
@@ -0,0 +1,96 @@
+/*
+ * 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.cassandra.spark.sparksql;
+
+import java.nio.ByteBuffer;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+import org.apache.spark.sql.catalyst.util.ArrayBasedMapData;
+import org.apache.spark.sql.catalyst.util.ArrayData;
+import org.apache.spark.unsafe.types.UTF8String;
+
+public class CellTombstonesInComplexDecorator extends RowBuilderDecorator
+{
+    private final int columnPos;
+    private final Map<String, ArrayData> tombstonedKeys = new LinkedHashMap<>();
+
+    public CellTombstonesInComplexDecorator(RowBuilder delegate)
+    {
+        super(delegate);
+        // Last item after this expansion is for the list of cell tombstones inside a complex data
+        columnPos = internalExpandRow();
+    }
+
+    @Override
+    protected int extraColumns()
+    {
+        return 1;
+    }
+
+    @Override
+    public void reset()
+    {
+        super.reset();
+        tombstonedKeys.clear();
+    }
+
+    @Override
+    public void onCell(Cell cell)
+    {
+        super.onCell(cell);
+        if (cell instanceof TombstonesInComplex)
+        {
+            TombstonesInComplex tombstones = (TombstonesInComplex) cell;
+
+            Object[] keys = tombstones.tombstonedKeys.stream()
+                                                           .map(ByteBuffer::array)
+                                                           .toArray();
+            tombstonedKeys.put(tombstones.columnName, ArrayData.toArrayData(keys));
+        }
+    }
+
+    @Override
+    public GenericInternalRow build()
+    {
+        // Append isUpdate flag
+        Object[] result = array();
+        if (!tombstonedKeys.isEmpty())
+        {
+            Object[] columns = new Object[tombstonedKeys.size()];
+            Object[] tombstones = new Object[tombstonedKeys.size()];
+            int index = 0;
+            for (Map.Entry<String, ArrayData> entry : tombstonedKeys.entrySet())
+            {
+                columns[index] = UTF8String.fromString(entry.getKey());
+                tombstones[index] = entry.getValue();
+                index++;
+            }
+            result[columnPos] = ArrayBasedMapData.apply(columns, tombstones);
+        }
+        else
+        {
+            result[columnPos] = null;
+        }
+
+        return super.build();
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/LastModifiedTimestampDecorator.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/LastModifiedTimestampDecorator.java
new file mode 100644
index 0000000..5414838
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/LastModifiedTimestampDecorator.java
@@ -0,0 +1,70 @@
+/*
+ * 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.cassandra.spark.sparksql;
+
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+
+/**
+ * Wrap a builder to append last modified timestamp
+ */
+public class LastModifiedTimestampDecorator extends RowBuilderDecorator
+{
+    private final int lmtColumnPosition;
+    private long lastModified = 0L;
+
+    public LastModifiedTimestampDecorator(RowBuilder delegate, String fieldName)
+    {
+        super(delegate);
+        int width = internalExpandRow();
+        int fieldIndex = fieldIndex(fieldName);
+        // Determine the last modified timestamp column position based on the query
+        lmtColumnPosition = fieldIndex >= 0 ? fieldIndex : width;
+    }
+
+    @Override
+    public void reset()
+    {
+        super.reset();
+        // Reset the lastModified the builder is re-used across rows
+        lastModified = 0L;
+    }
+
+    @Override
+    public void onCell(Cell cell)
+    {
+        super.onCell(cell);
+        lastModified = Math.max(lastModified, cell.timestamp);
+    }
+
+    @Override
+    protected int extraColumns()
+    {
+        return 1;
+    }
+
+    @Override
+    public GenericInternalRow build()
+    {
+        // Append last modified timestamp
+        Object[] result = array();
+        result[lmtColumnPosition] = lastModified;
+        return super.build();
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/NoMatchFoundException.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/NoMatchFoundException.java
new file mode 100644
index 0000000..b17bcd1
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/NoMatchFoundException.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.cassandra.spark.sparksql;
+
+public class NoMatchFoundException extends Exception
+{
+    public NoMatchFoundException()
+    {
+    }
+
+    public NoMatchFoundException(String message)
+    {
+        super(message);
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/RangeTombstone.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/RangeTombstone.java
new file mode 100644
index 0000000..adc8753
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/RangeTombstone.java
@@ -0,0 +1,33 @@
+/*
+ * 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.cassandra.spark.sparksql;
+
+import java.util.List;
+
+class RangeTombstone extends Tombstone
+{
+    public final List<RangeTombstoneMarker> rangeTombstoneMarkers;
+
+    RangeTombstone(Object[] values, long timestamp, List<RangeTombstoneMarker> markers)
+    {
+        super(values, timestamp);
+        rangeTombstoneMarkers = markers;
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/RangeTombstoneDecorator.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/RangeTombstoneDecorator.java
new file mode 100644
index 0000000..d3b4396
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/RangeTombstoneDecorator.java
@@ -0,0 +1,94 @@
+/*
+ * 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.cassandra.spark.sparksql;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+import org.apache.spark.sql.catalyst.util.ArrayData;
+
+public class RangeTombstoneDecorator extends RowBuilderDecorator
+{
+    private final int columnPosition;
+    private final List<InternalRow> rangeTombstoneList;
+
+    public RangeTombstoneDecorator(RowBuilder delegate)
+    {
+        super(delegate);
+        // Last item after this expansion is for the list of cell tombstones inside a complex data
+        this.columnPosition = internalExpandRow();
+        this.rangeTombstoneList = new ArrayList<>();
+    }
+
+    @Override
+    protected int extraColumns()
+    {
+        return 1;
+    }
+
+    @Override
+    public void reset()
+    {
+        super.reset();
+        rangeTombstoneList.clear();
+    }
+
+    @Override
+    public void onCell(Cell cell)
+    {
+        super.onCell(cell);
+
+        if (cell instanceof RangeTombstone)
+        {
+            RangeTombstone rangeTombstone = (RangeTombstone) cell;
+            // See SchemaFeatureSet.RANGE_DELETION#fieldDataType for the schema;
+            // each range has 4 fields: Start, StartInclusive, End, EndInclusive
+            Object[] range = null;
+            for (RangeTombstoneMarker marker : rangeTombstone.rangeTombstoneMarkers)
+            {
+                // `range` array is used to collect data from `marker` and is added into `rangeTombstoneList`
+                range = marker.computeRange(range, rangeTombstoneList, getCqlTable());
+            }
+            Preconditions.checkState(range == null, "Tombstone range should be closed");
+        }
+   }
+
+    @Override
+    public GenericInternalRow build()
+    {
+        // `result` array is declared in the `RowBuilder`, the decorator first expands the array,
+        // and adds the data to its allocated slot on build, `result` array is the data to produce a Spark row
+        Object[] result = array();
+        if (rangeTombstoneList.isEmpty())
+        {
+            result[columnPosition] = null;
+        }
+        else
+        {
+            result[columnPosition] = ArrayData.toArrayData(rangeTombstoneList.toArray());
+        }
+
+        return super.build();
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/RangeTombstoneMarker.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/RangeTombstoneMarker.java
new file mode 100644
index 0000000..63bae01
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/RangeTombstoneMarker.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.cassandra.spark.sparksql;
+
+import java.util.List;
+
+import org.apache.cassandra.spark.data.CqlTable;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+public interface RangeTombstoneMarker
+{
+    int TOTAL_FIELDS = 4;
+    int START_FIELD_POSITION = 0;
+    int START_INCLUSIVE_FIELD_POSITION = 1;
+    int END_FIELD_POSITION = 2;
+    int END_INCLUSIVE_FIELD_POSITION = 3;
+
+    boolean isBoundary();
+    boolean isOpen(boolean value);
+    boolean isClose(boolean value);
+
+    long openDeletionTime(boolean value);
+    long closeDeletionTime(boolean value);
+
+    @Nullable
+    Object[] computeRange(@Nullable Object[] range,
+                          @NotNull List<InternalRow> list,
+                          @NotNull CqlTable table);
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/RowBuilder.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/RowBuilder.java
new file mode 100644
index 0000000..e6cc452
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/RowBuilder.java
@@ -0,0 +1,58 @@
+/*
+ * 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.cassandra.spark.sparksql;
+
+import org.apache.cassandra.spark.data.CqlTable;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+
+public interface RowBuilder
+{
+    CqlTable getCqlTable();
+
+    void reset();
+
+    boolean isFirstCell();
+
+    boolean hasMoreCells();
+
+    void onCell(Cell cell);
+
+    void copyKeys(Cell cell);
+
+    void copyValue(Cell cell);
+
+    Object[] array();
+
+    int columnsCount();
+
+    boolean hasRegularValueColumn();
+
+    int fieldIndex(String name);
+
+    /**
+     * Expand the row with more columns. The extra columns are appended to the row.
+     *
+     * @param extraColumns number of columns to append
+     * @return length of row before expanding
+     */
+    int expandRow(int extraColumns);
+
+    GenericInternalRow build();
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/RowBuilderDecorator.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/RowBuilderDecorator.java
new file mode 100644
index 0000000..7dfbc59
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/RowBuilderDecorator.java
@@ -0,0 +1,123 @@
+/*
+ * 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.cassandra.spark.sparksql;
+
+import org.apache.cassandra.spark.data.CqlTable;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+
+abstract class RowBuilderDecorator implements RowBuilder
+{
+    protected final RowBuilder delegate;
+
+    RowBuilderDecorator(RowBuilder delegate)
+    {
+        this.delegate = delegate;
+    }
+
+    @Override
+    public int columnsCount()
+    {
+        return delegate.columnsCount();
+    }
+
+    @Override
+    public boolean hasRegularValueColumn()
+    {
+        return delegate.hasRegularValueColumn();
+    }
+
+    @Override
+    public void reset()
+    {
+        delegate.reset();
+    }
+
+    @Override
+    public boolean isFirstCell()
+    {
+        return delegate.isFirstCell();
+    }
+
+    @Override
+    public boolean hasMoreCells()
+    {
+        return delegate.hasMoreCells();
+    }
+
+    @Override
+    public void onCell(Cell cell)
+    {
+        delegate.onCell(cell);
+    }
+
+    @Override
+    public void copyKeys(Cell cell)
+    {
+        delegate.copyKeys(cell);
+    }
+
+    @Override
+    public void copyValue(Cell cell)
+    {
+        delegate.copyValue(cell);
+    }
+
+    @Override
+    public Object[] array()
+    {
+        return delegate.array();
+    }
+
+    @Override
+    public int expandRow(int extraColumns)
+    {
+        return delegate.expandRow(extraColumns + extraColumns()) + extraColumns();
+    }
+
+    @Override
+    public CqlTable getCqlTable()
+    {
+        return delegate.getCqlTable();
+    }
+
+    /**
+     * Preferred to call if the decorator is adding extra columns
+     *
+     * @return the index of the fist extra column
+     */
+    protected int internalExpandRow()
+    {
+        return expandRow(0) - extraColumns();
+    }
+
+    protected abstract int extraColumns();
+
+    @Override
+    public int fieldIndex(String name)
+    {
+        return delegate.fieldIndex(name);
+    }
+
+    @Override
+    public GenericInternalRow build()
+    {
+        return delegate.build();
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/Tombstone.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/Tombstone.java
new file mode 100644
index 0000000..c14872a
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/Tombstone.java
@@ -0,0 +1,35 @@
+/*
+ * 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.cassandra.spark.sparksql;
+
+class Tombstone extends Cell
+{
+    Tombstone(Object[] values, long timestamp)
+    {
+        // Using -1 for position; since we do not copy cell value from row deletion, position should not be used
+        super(values, -1, true, false, timestamp);
+    }
+
+    @Override
+    boolean isTombstone()
+    {
+        return true;
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/TombstonesInComplex.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/TombstonesInComplex.java
new file mode 100644
index 0000000..2fd4211
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/TombstonesInComplex.java
@@ -0,0 +1,41 @@
+/*
+ * 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.cassandra.spark.sparksql;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+class TombstonesInComplex extends Cell
+{
+    public final List<ByteBuffer> tombstonedKeys;
+    public final String columnName;
+
+    TombstonesInComplex(Object[] values,
+                        int position,
+                        boolean isNewRow,
+                        long timestamp,
+                        String columnName,
+                        List<ByteBuffer> tombstonedKeys)
+    {
+        super(values, position, isNewRow, false, timestamp);
+        this.columnName = columnName;
+        this.tombstonedKeys = tombstonedKeys;
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/UpdateFlagDecorator.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/UpdateFlagDecorator.java
new file mode 100644
index 0000000..4415fc3
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/UpdateFlagDecorator.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.cassandra.spark.sparksql;
+
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+
+public class UpdateFlagDecorator extends RowBuilderDecorator
+{
+    private final int columnPosition;
+    private boolean isUpdate = false;
+
+    public UpdateFlagDecorator(RowBuilder delegate)
+    {
+        super(delegate);
+        // Last item after this expansion is for the liveness column
+        this.columnPosition = internalExpandRow();
+    }
+
+    @Override
+    public void reset()
+    {
+        super.reset();
+        isUpdate = false;
+    }
+
+    @Override
+    public void onCell(Cell cell)
+    {
+        super.onCell(cell);
+        isUpdate = cell.isUpdate;
+    }
+
+    @Override
+    protected int extraColumns()
+    {
+        return 1;
+    }
+
+    @Override
+    public GenericInternalRow build()
+    {
+        // Append isUpdate flag
+        Object[] result = array();
+        result[columnPosition] = isUpdate;
+        return super.build();
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/UpdatedFieldsIndicatorDecorator.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/UpdatedFieldsIndicatorDecorator.java
new file mode 100644
index 0000000..fab8767
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/UpdatedFieldsIndicatorDecorator.java
@@ -0,0 +1,75 @@
+/*
+ * 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.cassandra.spark.sparksql;
+
+import java.util.BitSet;
+
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+
+public class UpdatedFieldsIndicatorDecorator extends RowBuilderDecorator
+{
+    private final int indicatorPosition;
+    private final BitSet appearedColumns;
+
+    public UpdatedFieldsIndicatorDecorator(RowBuilder delegate)
+    {
+        super(delegate);
+        // Last item after this expansion is for the indicator column
+        this.indicatorPosition = internalExpandRow();
+        this.appearedColumns = new BitSet(columnsCount());
+    }
+
+    @Override
+    protected int extraColumns()
+    {
+        return 1;
+    }
+
+    @Override
+    public void copyKeys(Cell cell)
+    {
+        super.copyKeys(cell);
+        int length = hasRegularValueColumn() && !cell.isTombstone() ? cell.values.length - 1 : cell.values.length;
+        appearedColumns.set(0, length);
+    }
+
+    @Override
+    public void copyValue(Cell cell)
+    {
+        super.copyValue(cell);
+        appearedColumns.set(cell.position);
+    }
+
+    @Override
+    public void reset()
+    {
+        super.reset();
+        appearedColumns.clear();
+    }
+
+    @Override
+    public GenericInternalRow build()
+    {
+        // Append updated fields indicator
+        Object[] result = array();
+        result[indicatorPosition] = appearedColumns.toByteArray();
+        return super.build();
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/filters/CdcOffset.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/filters/CdcOffset.java
new file mode 100644
index 0000000..9b6d414
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/filters/CdcOffset.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.cassandra.spark.sparksql.filters;
+
+import java.io.IOException;
+import java.io.Serializable;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.commons.lang.builder.HashCodeBuilder;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.spark.sql.execution.streaming.Offset;
+import org.jetbrains.annotations.NotNull;
+
+public class CdcOffset extends Offset implements Serializable, Comparable<CdcOffset>
+{
+    @VisibleForTesting
+    public static final ObjectMapper MAPPER = new ObjectMapper();
+
+    private final long timestampMicros;
+
+    @JsonCreator
+    public CdcOffset(@JsonProperty("timestamp") long timestampMicros)
+    {
+        this.timestampMicros = timestampMicros;
+    }
+
+    public static CdcOffset fromJson(String json)
+    {
+        try
+        {
+            return MAPPER.readValue(json, CdcOffset.class);
+        }
+        catch (IOException exception)
+        {
+            throw new RuntimeException(exception);
+        }
+    }
+
+    public long getTimestampMicros()
+    {
+        return timestampMicros;
+    }
+
+    @Override
+    public String json()
+    {
+        try
+        {
+            return MAPPER.writeValueAsString(this);
+        }
+        catch (IOException exception)
+        {
+            throw new RuntimeException(exception);
+        }
+    }
+
+    @Override
+    public String toString()
+    {
+        return json();
+    }
+
+    @Override
+    public boolean equals(Object other)
+    {
+        if (other == null)
+        {
+            return false;
+        }
+        if (this == other)
+        {
+            return true;
+        }
+        if (this.getClass() != other.getClass())
+        {
+            return false;
+        }
+
+        CdcOffset that = (CdcOffset) other;
+        return timestampMicros == that.timestampMicros;
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return new HashCodeBuilder()
+               .append(timestampMicros)
+               .toHashCode();
+    }
+
+    @Override
+    public int compareTo(@NotNull CdcOffset that)
+    {
+        return Long.compare(this.timestampMicros, that.timestampMicros);
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/filters/CdcOffsetFilter.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/filters/CdcOffsetFilter.java
new file mode 100644
index 0000000..8c6143d
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/filters/CdcOffsetFilter.java
@@ -0,0 +1,91 @@
+/*
+ * 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.cassandra.spark.sparksql.filters;
+
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+import java.time.Duration;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.cassandra.bridge.TokenRange;
+import org.apache.cassandra.spark.reader.SparkSSTableReader;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * CDC offset filter used to checkpoint Streaming queries and only read mutations within watermark window
+ */
+public final class CdcOffsetFilter implements Serializable
+{
+    @NotNull
+    private final CdcOffset start;
+    private final long maxAgeMicros;
+
+    private CdcOffsetFilter(@NotNull CdcOffset start, @NotNull Duration watermarkWindow)
+    {
+        Preconditions.checkNotNull(start, "Start offset cannot be null");
+        this.start = start;
+        this.maxAgeMicros = start.getTimestampMicros() - TimeUnit.NANOSECONDS.toMicros(watermarkWindow.toNanos());
+    }
+
+    /**
+     * Return true if mutation timestamp overlaps with watermark window
+     *
+     * @param timestampMicros mutation timestamp in micros
+     * @return true if timestamp overlaps with range
+     */
+    public boolean overlaps(long timestampMicros)
+    {
+        return timestampMicros >= maxAgeMicros;
+    }
+
+    @NotNull
+    public static CdcOffsetFilter of(@NotNull CdcOffset start, @NotNull Duration watermarkWindow)
+    {
+        return new CdcOffsetFilter(start, watermarkWindow);
+    }
+
+    public long maxAgeMicros()
+    {
+        return maxAgeMicros;
+    }
+
+    @NotNull
+    public CdcOffset start()
+    {
+        return start;
+    }
+
+    public boolean overlaps(TokenRange tokenRange)
+    {
+        return false;
+    }
+
+    public boolean filter(ByteBuffer key)
+    {
+        return false;
+    }
+
+    public boolean filter(SparkSSTableReader reader)
+    {
+        return false;
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/filters/PartitionKeyFilter.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/filters/PartitionKeyFilter.java
new file mode 100644
index 0000000..fb986b2
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/filters/PartitionKeyFilter.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.cassandra.spark.sparksql.filters;
+
+import java.io.Serializable;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.cassandra.bridge.TokenRange;
+import org.apache.spark.util.SerializableBuffer;
+import org.jetbrains.annotations.NotNull;
+
+public final class PartitionKeyFilter implements Serializable
+{
+    @NotNull
+    private final SerializableBuffer key;
+    @NotNull
+    private final BigInteger token;
+
+    private PartitionKeyFilter(@NotNull ByteBuffer filterKey, @NotNull BigInteger filterKeyTokenValue)
+    {
+        key = new SerializableBuffer(filterKey);
+        token = filterKeyTokenValue;
+    }
+
+    @NotNull
+    public TokenRange tokenRange()
+    {
+        return TokenRange.singleton(token);
+    }
+
+    @NotNull
+    public ByteBuffer key()
+    {
+        return key.buffer();
+    }
+
+    @NotNull
+    public BigInteger token()
+    {
+        return token;
+    }
+
+    public boolean overlaps(@NotNull TokenRange tokenRange)
+    {
+        return tokenRange.contains(token);
+    }
+
+    public boolean matches(@NotNull ByteBuffer key)
+    {
+        return key.compareTo(this.key.buffer()) == 0;
+    }
+
+    public boolean filter(@NotNull ByteBuffer key)
+    {
+        return this.key.buffer().compareTo(key) == 0;
+    }
+
+    @NotNull
+    public static PartitionKeyFilter create(@NotNull ByteBuffer filterKey, @NotNull BigInteger filterKeyTokenValue)
+    {
+        Preconditions.checkArgument(filterKey.capacity() != 0);
+        return new PartitionKeyFilter(filterKey, filterKeyTokenValue);
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/filters/PruneColumnFilter.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/filters/PruneColumnFilter.java
new file mode 100644
index 0000000..c032659
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/filters/PruneColumnFilter.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.cassandra.spark.sparksql.filters;
+
+import java.util.Set;
+
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Prune column push-down filter to skip reading columns that are not needed
+ */
+public class PruneColumnFilter
+{
+    private final Set<String> requiredColumns;
+
+    public PruneColumnFilter(@NotNull Set<String> requiredColumns)
+    {
+        this.requiredColumns = requiredColumns;
+    }
+
+    public Set<String> requiredColumns()
+    {
+        return requiredColumns;
+    }
+
+    public int size()
+    {
+        return requiredColumns.size();
+    }
+
+    public boolean includeColumn(String columnName)
+    {
+        return requiredColumns.contains(columnName);
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/filters/SparkRangeFilter.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/filters/SparkRangeFilter.java
new file mode 100644
index 0000000..2467f9a
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/sparksql/filters/SparkRangeFilter.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.cassandra.spark.sparksql.filters;
+
+import java.io.Serializable;
+import java.math.BigInteger;
+
+import org.apache.cassandra.bridge.TokenRange;
+import org.jetbrains.annotations.NotNull;
+
+public final class SparkRangeFilter implements Serializable
+{
+    @NotNull
+    private final TokenRange tokenRange;
+
+    private SparkRangeFilter(@NotNull TokenRange tokenRange)
+    {
+        this.tokenRange = tokenRange;
+    }
+
+    @NotNull
+    public TokenRange tokenRange()
+    {
+        return tokenRange;
+    }
+
+    public boolean overlaps(TokenRange tokenRange)
+    {
+        return this.tokenRange.isConnected(tokenRange);
+    }
+
+    public boolean overlaps(BigInteger token)
+    {
+        return tokenRange.contains(token);
+    }
+
+    public boolean skipPartition(BigInteger token)
+    {
+        return !tokenRange.contains(token);
+    }
+
+    @NotNull
+    public static SparkRangeFilter create(@NotNull TokenRange tokenRange)
+    {
+        return new SparkRangeFilter(tokenRange);
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/stats/Stats.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/stats/Stats.java
new file mode 100644
index 0000000..45e5400
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/stats/Stats.java
@@ -0,0 +1,667 @@
+/*
+ * 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.cassandra.spark.stats;
+
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.cassandra.spark.cdc.IPartitionUpdateWrapper;
+import org.apache.cassandra.spark.data.CqlField;
+import org.apache.cassandra.spark.data.SSTable;
+import org.apache.cassandra.spark.data.SSTablesSupplier;
+import org.apache.cassandra.spark.sparksql.filters.PartitionKeyFilter;
+import org.apache.cassandra.spark.sparksql.filters.SparkRangeFilter;
+import org.apache.cassandra.spark.utils.streaming.SSTableSource;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+public abstract class Stats
+{
+
+    public static class DoNothingStats extends Stats
+    {
+        public static final DoNothingStats INSTANCE = new DoNothingStats();
+    }
+
+    // Spark Row Iterator
+
+    /**
+     * On open SparkRowIterator
+     */
+    public void openedSparkRowIterator()
+    {
+    }
+
+    /**
+     * On iterate to next row
+     */
+    public void nextRow()
+    {
+    }
+
+    /**
+     * Open closed SparkRowIterator
+     *
+     * @param timeOpenNanos time SparkRowIterator was open in nanos
+     */
+    public void closedSparkRowIterator(long timeOpenNanos)
+    {
+    }
+
+    // Spark Cell Iterator
+
+    /**
+     * On opened SparkCellIterator
+     */
+    public void openedSparkCellIterator()
+    {
+    }
+
+    /**
+     * On iterate to next cell
+     *
+     * @param timeNanos time since last cell
+     */
+    public void nextCell(long timeNanos)
+    {
+    }
+
+    /**
+     * How long it took to deserialize a particular field
+     *
+     * @param field     CQL field
+     * @param timeNanos time to deserialize in nanoseconds
+     */
+    public void fieldDeserialization(CqlField field, long timeNanos)
+    {
+    }
+
+    /**
+     * SSTableReader skipped partition in SparkCellIterator e.g. because out-of-range
+     *
+     * @param key   partition key
+     * @param token partition key token
+     */
+    public void skippedPartitionInIterator(ByteBuffer key, BigInteger token)
+    {
+    }
+
+    /**
+     * On closed SparkCellIterator
+     *
+     * @param timeOpenNanos time SparkCellIterator was open in nanos
+     */
+    public void closedSparkCellIterator(long timeOpenNanos)
+    {
+    }
+
+    // Partitioned Data Layer
+
+    /**
+     * Failed to open SSTable reads for a replica
+     *
+     * @param replica   the replica
+     * @param throwable the exception
+     */
+    public <T extends SSTablesSupplier> void failedToOpenReplica(T replica, Throwable throwable)
+    {
+    }
+
+    /**
+     * Failed to open SSTableReaders for enough replicas to satisfy the consistency level
+     *
+     * @param primaryReplicas primary replicas selected
+     * @param backupReplicas  backup replicas selected
+     */
+    public <T extends SSTablesSupplier> void notEnoughReplicas(Set<T> primaryReplicas, Set<T> backupReplicas)
+    {
+    }
+
+    /**
+     * Open SSTableReaders for enough replicas to satisfy the consistency level
+     *
+     * @param primaryReplicas primary replicas selected
+     * @param backupReplicas  backup replicas selected
+     * @param timeNanos       time in nanoseconds
+     */
+    public <T extends SSTablesSupplier> void openedReplicas(Set<T> primaryReplicas,
+                                                            Set<T> backupReplicas,
+                                                            long timeNanos)
+    {
+    }
+
+    // CDC
+
+    public void insufficientReplicas(IPartitionUpdateWrapper update, int numCopies, int minimumReplicasPerMutation)
+    {
+    }
+
+    public void lateMutationPublished(IPartitionUpdateWrapper update)
+    {
+    }
+
+    public void publishedMutation(IPartitionUpdateWrapper update)
+    {
+    }
+
+    /**
+     * The time taken to list the snapshot
+     *
+     * @param replica   the replica
+     * @param timeNanos time in nanoseconds to list the snapshot
+     */
+    public <T extends SSTablesSupplier> void timeToListSnapshot(T replica, long timeNanos)
+    {
+    }
+
+    // CompactionScanner
+
+    /**
+     * On opened CompactionScanner
+     *
+     * @param timeToOpenNanos time to open the CompactionScanner in nanos
+     */
+    public void openedCompactionScanner(long timeToOpenNanos)
+    {
+    }
+
+    // SSTable Data.db Input Stream
+
+    /**
+     * On open an input stream on a Data.db file
+     */
+    public void openedDataInputStream()
+    {
+    }
+
+    /**
+     * On skip bytes from an input stream on a Data.db file,
+     * mostly from SSTableReader skipping out of range partition
+     */
+    public void skippedBytes(long length)
+    {
+    }
+
+    /**
+     * The SSTableReader used the Summary.db/Index.db offsets to skip to the first in-range partition
+     * skipping 'length' bytes before reading the Data.db file
+     */
+    public void skippedDataDbStartOffset(long length)
+    {
+    }
+
+    /**
+     * The SSTableReader used the Summary.db/Index.db offsets to close after passing the last in-range partition
+     * after reading 'length' bytes from the Data.db file
+     */
+    public void skippedDataDbEndOffset(long length)
+    {
+    }
+
+    /**
+     * On read bytes from an input stream on a Data.db file
+     */
+    public void readBytes(int length)
+    {
+    }
+
+    /**
+     * On decompress bytes from an input stream on a compressed Data.db file
+     *
+     * @param compressedLen   compressed length in bytes
+     * @param decompressedLen compressed length in bytes
+     */
+    public void decompressedBytes(int compressedLen, int decompressedLen)
+    {
+    }
+
+    /**
+     * On an exception when decompressing an SSTable e.g. if corrupted
+     *
+     * @param ssTable   the SSTable being decompressed
+     * @param throwable the exception thrown
+     */
+    public void decompressionException(SSTable ssTable, Throwable throwable)
+    {
+    }
+
+    /**
+     * On close an input stream on a Data.db file
+     */
+    public void closedDataInputStream()
+    {
+    }
+
+    // Partition Push-Down Filters
+
+    /**
+     * Partition key push-down filter skipped SSTable because Filter.db did not contain partition
+     */
+    public void missingInBloomFilter()
+    {
+    }
+
+    /**
+     * Partition key push-down filter skipped SSTable because Index.db did not contain partition
+     */
+    public void missingInIndex()
+    {
+    }
+
+    // SSTable Filters
+
+    /**
+     * SSTableReader skipped SSTable e.g. because not overlaps with Spark worker token range
+     *
+     * @param sparkRangeFilter    spark range filter used to filter SSTable
+     * @param partitionKeyFilters list of partition key filters used to filter SSTable
+     * @param firstToken          SSTable first token
+     * @param lastToken           SSTable last token
+     */
+    public void skippedSSTable(@Nullable SparkRangeFilter sparkRangeFilter,
+                               @NotNull List<PartitionKeyFilter> partitionKeyFilters,
+                               @NotNull BigInteger firstToken,
+                               @NotNull BigInteger lastToken)
+    {
+    }
+
+    /**
+     * SSTableReader skipped an SSTable because it is repaired and the Spark worker is not the primary repair replica
+     *
+     * @param ssTable    the SSTable being skipped
+     * @param repairedAt last repair timestamp for SSTable
+     */
+    public void skippedRepairedSSTable(SSTable ssTable, long repairedAt)
+    {
+    }
+
+    /**
+     * SSTableReader skipped partition e.g. because out-of-range
+     *
+     * @param key   partition key
+     * @param token partition key token
+     */
+    public void skippedPartition(ByteBuffer key, BigInteger token)
+    {
+    }
+
+    /**
+     * SSTableReader opened an SSTable
+     *
+     * @param timeNanos total time to open in nanoseconds
+     */
+    public void openedSSTable(SSTable ssTable, long timeNanos)
+    {
+    }
+
+    /**
+     * SSTableReader opened and deserialized a Summary.db file
+     *
+     * @param timeNanos total time to read in nanoseconds
+     */
+    public void readSummaryDb(SSTable ssTable, long timeNanos)
+    {
+    }
+
+    /**
+     * SSTableReader opened and deserialized a Index.db file
+     *
+     * @param timeNanos total time to read in nanoseconds
+     */
+    public void readIndexDb(SSTable ssTable, long timeNanos)
+    {
+    }
+
+    /**
+     * Read a single partition in the Index.db file
+     *
+     * @param key   partition key
+     * @param token partition key token
+     */
+    public void readPartitionIndexDb(ByteBuffer key, BigInteger token)
+    {
+    }
+
+    /**
+     * SSTableReader read next partition
+     *
+     * @param timeOpenNanos time in nanoseconds since last partition was read
+     */
+    public void nextPartition(long timeOpenNanos)
+    {
+    }
+
+    /**
+     * Exception thrown when reading SSTable
+     *
+     * @param throwable exception thrown
+     * @param keyspace  keyspace
+     * @param table     table
+     * @param ssTable   the SSTable being read
+     */
+    public void corruptSSTable(Throwable throwable, String keyspace, String table, SSTable ssTable)
+    {
+    }
+
+    /**
+     * SSTableReader closed an SSTable
+     *
+     * @param timeOpenNanos time in nanoseconds SSTable was open
+     */
+    public void closedSSTable(long timeOpenNanos)
+    {
+    }
+
+    // SSTable Input Stream
+
+    /**
+     * When {@link org.apache.cassandra.spark.utils.streaming.SSTableInputStream} queue is full, usually indicating
+     * job is CPU-bound and blocked on the CompactionIterator
+     *
+     * @param ssTable the SSTable source for this input stream
+     */
+    public void inputStreamQueueFull(SSTableSource<? extends SSTable> ssTable)
+    {
+    }
+
+    /**
+     * Failure occurred in the {@link org.apache.cassandra.spark.utils.streaming.SSTableInputStream}
+     *
+     * @param ssTable   the SSTable source for this input stream
+     * @param throwable throwable
+     */
+    public void inputStreamFailure(SSTableSource<? extends SSTable> ssTable, Throwable throwable)
+    {
+    }
+
+    /**
+     * Time the {@link org.apache.cassandra.spark.utils.streaming.SSTableInputStream} spent blocking on queue
+     * waiting for bytes. High time spent blocking indicates the job is network-bound, or blocked on the
+     * {@link org.apache.cassandra.spark.utils.streaming.SSTableSource} to supply the bytes.
+     *
+     * @param ssTable the SSTable source for this input stream
+     * @param nanos   time in nanoseconds
+     */
+    public void inputStreamTimeBlocked(SSTableSource<? extends SSTable> ssTable, long nanos)
+    {
+    }
+
+    /**
+     * Bytes written to {@link org.apache.cassandra.spark.utils.streaming.SSTableInputStream}
+     * by the {@link org.apache.cassandra.spark.utils.streaming.SSTableSource}
+     *
+     * @param ssTable the SSTable source for this input stream
+     * @param length  number of bytes written
+     */
+    public void inputStreamBytesWritten(SSTableSource<? extends SSTable> ssTable, int length)
+    {
+    }
+
+    /**
+     * Bytes read from {@link org.apache.cassandra.spark.utils.streaming.SSTableInputStream}
+     *
+     * @param ssTable         the SSTable source for this input stream
+     * @param length          number of bytes read
+     * @param queueSize       current queue size
+     * @param percentComplete % completion
+     */
+    public void inputStreamByteRead(SSTableSource<? extends SSTable> ssTable,
+                                    int length,
+                                    int queueSize,
+                                    int percentComplete)
+    {
+    }
+
+    /**
+     * {@link org.apache.cassandra.spark.utils.streaming.SSTableSource} has finished writing
+     * to {@link org.apache.cassandra.spark.utils.streaming.SSTableInputStream} after reaching expected file length
+     *
+     * @param ssTable the SSTable source for this input stream
+     */
+    public void inputStreamEndBuffer(SSTableSource<? extends SSTable> ssTable)
+    {
+    }
+
+    /**
+     * {@link org.apache.cassandra.spark.utils.streaming.SSTableInputStream} finished and closed
+     *
+     * @param ssTable           the SSTable source for this input stream
+     * @param runTimeNanos      total time open in nanoseconds
+     * @param totalNanosBlocked total time blocked on queue waiting for bytes in nanoseconds
+     */
+    public void inputStreamEnd(SSTableSource<? extends SSTable> ssTable, long runTimeNanos, long totalNanosBlocked)
+    {
+    }
+
+    /**
+     * Called when the InputStream skips bytes
+     *
+     * @param ssTable         the SSTable source for this input stream
+     * @param bufferedSkipped the number of bytes already buffered in memory skipped
+     * @param rangeSkipped    the number of bytes skipped
+     *                        by efficiently incrementing the start range for the next request
+     */
+    public void inputStreamBytesSkipped(SSTableSource<? extends SSTable> ssTable,
+                                        long bufferedSkipped,
+                                        long rangeSkipped)
+    {
+    }
+
+    /**
+     * Number of successfully read mutations
+     *
+     * @param incrCount delta value to add to the count
+     */
+    public void mutationsReadCount(long incrCount)
+    {
+    }
+
+    /**
+     * Deserialized size of a successfully read mutation
+     *
+     * @param nBytes mutation size in bytes
+     */
+    public void mutationsReadBytes(long nBytes)
+    {
+    }
+
+    /**
+     * Called when received a mutation with unknown table
+     *
+     * @param incrCount delta value to add to the count
+     */
+    public void mutationsIgnoredUnknownTableCount(long incrCount)
+    {
+    }
+
+    /**
+     * Called when deserialization of a mutation fails
+     *
+     * @param incrCount delta value to add to the count
+     */
+    public void mutationsDeserializeFailedCount(long incrCount)
+    {
+    }
+
+    /**
+     * Called when a mutation's checksum calculation fails or doesn't match with expected checksum
+     *
+     * @param incrCount delta value to add to the count
+     */
+    public void mutationsChecksumMismatchCount(long incrCount)
+    {
+    }
+
+    /**
+     * Called when a mutation doesn't have expected table id, and ignored from processing
+     *
+     * @param incrCount delta value to add to the count
+     */
+    public void mutationsIgnoredUntrackedTableCount(long incrCount)
+    {
+    }
+
+    /**
+     * Called when a mutation doesn't have expected token range, and ignored from processing
+     *
+     * @param incrCount delta value to add to the count
+     */
+    public void mutationsIgnoredOutOfTokenRangeCount(long incrCount)
+    {
+    }
+
+    /**
+     * Time taken to read a CommitLog file
+     *
+     * @param timeTaken time taken, in nano secs
+     */
+    public void commitLogReadTime(long timeTaken)
+    {
+    }
+
+    /**
+     * Number of mutations read by a micro batch
+     *
+     * @param count mutations count
+     */
+    public void mutationsReadPerBatch(long count)
+    {
+    }
+
+    /**
+     * Time taken by a micro batch, i.e, to read CommitLog files of a batch
+     *
+     * @param timeTaken time taken, in nano secs
+     */
+    public void mutationsBatchReadTime(long timeTaken)
+    {
+    }
+
+    /**
+     * Time taken to aggregate and filter mutations
+     *
+     * @param timeTakenNanos time taken in nanoseconds
+     */
+    public void mutationsFilterTime(long timeTakenNanos)
+    {
+    }
+
+    /**
+     * Difference between the time mutation was created and time the same was read by a spark worker
+     *
+     * @param latency time difference, in milli secs
+     */
+    public void mutationReceivedLatency(long latency)
+    {
+    }
+
+    /**
+     * Difference between the time mutation was created and time the same produced as a spark row
+     *
+     * @param latency time difference, in milli secs
+     */
+    public void mutationProducedLatency(long latency)
+    {
+    }
+
+    /**
+     * Number of unexpected CommitLog EOF occurrences
+     *
+     * @param incrCount delta value to add to the count
+     */
+    public void commitLogSegmentUnexpectedEndErrorCount(long incrCount)
+    {
+    }
+
+    /**
+     * Number of invalid mutation size occurrences
+     *
+     * @param incrCount delta value to add to the count
+     */
+    public void commitLogInvalidSizeMutationCount(long incrCount)
+    {
+    }
+
+    /**
+     * Number of IO exceptions seen while reading CommitLog header
+     *
+     * @param incrCount delta value to add to the count
+     */
+    public void commitLogHeaderReadFailureCount(long incrCount)
+    {
+    }
+
+    /**
+     * Time taken to read a CommitLog's header
+     *
+     * @param timeTaken time taken, in nano secs
+     */
+    public void commitLogHeaderReadTime(long timeTaken)
+    {
+    }
+
+    /**
+     * Time taken to read a CommitLog's segment/section
+     *
+     * @param timeTaken time taken, in nano secs
+     */
+    public void commitLogSegmentReadTime(long timeTaken)
+    {
+    }
+
+    /**
+     * Number of CommitLogs skipped
+     *
+     * @param incrCount delta value to add to the count
+     */
+    public void skippedCommitLogsCount(long incrCount)
+    {
+    }
+
+    /**
+     * Number of bytes skipped/seeked when reading the CommitLog
+     *
+     * @param nBytes number of bytes
+     */
+    public void commitLogBytesSkippedOnRead(long nBytes)
+    {
+    }
+
+    /**
+     * Number of CommitLog bytes fetched
+     *
+     * @param nBytes number of bytes
+     */
+    public void commitLogBytesFetched(long nBytes)
+    {
+    }
+
+    /**
+     * The {@code org.apache.cassandra.db.commitlog.BufferingCommitLogReader} dropped a mutation because the client
+     * write timestamp exceeded the watermarker timestamp window
+     *
+     * @param maxTimestampMicros mutation max timestamp in microseconds
+     */
+    public void droppedOldMutation(long maxTimestampMicros)
+    {
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/ArrayUtils.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/ArrayUtils.java
new file mode 100644
index 0000000..d302edf
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/ArrayUtils.java
@@ -0,0 +1,42 @@
+/*
+ * 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.cassandra.spark.utils;
+
+import com.google.common.base.Preconditions;
+
+public final class ArrayUtils
+{
+    private ArrayUtils()
+    {
+        throw new IllegalStateException(getClass() + " is static utility class and shall not be instantiated");
+    }
+
+    public static Object[] retain(Object[] source,  int index,  int length)
+    {
+        Preconditions.checkArgument(source != null && 0 <= index && 0 <= length);
+        Preconditions.checkArgument(index + length <= source.length, "Requested retain range exceed the source array!");
+        Object[] result = new Object[length];
+        if (length > 0)
+        {
+            System.arraycopy(source, index, result, 0, length);
+        }
+        return result;
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/ByteBufferUtils.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/ByteBufferUtils.java
new file mode 100644
index 0000000..32fc682
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/ByteBufferUtils.java
@@ -0,0 +1,225 @@
+/*
+ * 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.cassandra.spark.utils;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataInput;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.Buffer;
+import java.nio.ByteBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.CharsetDecoder;
+import java.nio.charset.StandardCharsets;
+import java.util.Arrays;
+
+import io.netty.util.concurrent.FastThreadLocal;
+
+public final class ByteBufferUtils
+{
+    private static final String EMPTY_STRING = "";
+    private static final FastThreadLocal<CharsetDecoder> UTF8_DECODER = new FastThreadLocal<CharsetDecoder>()
+    {
+        @Override
+        protected CharsetDecoder initialValue()
+        {
+            return StandardCharsets.UTF_8.newDecoder();
+        }
+    };
+    private static final char[] HEX_ARRAY = "0123456789ABCDEF".toCharArray();
+
+    private ByteBufferUtils()
+    {
+        throw new IllegalStateException(getClass() + " is static utility class and shall not be instantiated");
+    }
+
+    public static void skipBytesFully(DataInput in, int bytes) throws IOException
+    {
+        int total = 0;
+        while (total < bytes)
+        {
+            int skipped = in.skipBytes(bytes - total);
+            if (skipped == 0)
+            {
+                throw new EOFException("EOF after " + total + " bytes out of " + bytes);
+            }
+            total += skipped;
+        }
+    }
+
+    public static byte[] readRemainingBytes(InputStream in, int size) throws IOException
+    {
+        ByteArrayOutputStream out = new ByteArrayOutputStream(size);
+        byte[] bytes = new byte[size];
+        int length;
+        while ((length = in.read(bytes)) > 0)
+        {
+            out.write(bytes, 0, length);
+        }
+        return out.toByteArray();
+    }
+
+    public static byte[] getArray(ByteBuffer buffer)
+    {
+        int length = buffer.remaining();
+
+        if (buffer.hasArray())
+        {
+            int boff = buffer.arrayOffset() + buffer.position();
+            return Arrays.copyOfRange(buffer.array(), boff, boff + length);
+        }
+        // Else, DirectByteBuffer.get() is the fastest route
+        byte[] bytes = new byte[length];
+        buffer.duplicate().get(bytes);
+
+        return bytes;
+    }
+
+    public static String stringThrowRuntime(ByteBuffer buffer)
+    {
+        try
+        {
+            return ByteBufferUtils.string(buffer);
+        }
+        catch (CharacterCodingException exception)
+        {
+            throw new RuntimeException(exception);
+        }
+    }
+
+    public static String string(ByteBuffer buffer) throws CharacterCodingException
+    {
+        if (buffer.remaining() <= 0)
+        {
+            return EMPTY_STRING;
+        }
+        return UTF8_DECODER.get().decode(buffer.duplicate()).toString();
+    }
+
+    private static String toHexString(byte[] bytes, int length)
+    {
+        return toHexString(bytes, 0, length);
+    }
+
+    static String toHexString(byte[] bytes, int offset, int length)
+    {
+        char[] hexCharacters = new char[length << 1];
+
+        int decimalValue;
+        for (int index = offset; index < offset + length; index++)
+        {
+            // Calculate the int value represented by the byte
+            decimalValue = bytes[index] & 0xFF;
+            // Retrieve hex character for 4 upper bits
+            hexCharacters[(index - offset) << 1] = HEX_ARRAY[decimalValue >> 4];
+            // Retrieve hex character for 4 lower bits
+            hexCharacters[((index - offset) << 1) + 1] = HEX_ARRAY[decimalValue & 0xF];
+        }
+
+        return new String(hexCharacters);
+    }
+
+    public static String toHexString(ByteBuffer buffer)
+    {
+        if (buffer == null)
+        {
+            return "null";
+        }
+
+        if (buffer.isReadOnly())
+        {
+            byte[] bytes = new byte[buffer.remaining()];
+            buffer.slice().get(bytes);
+            return ByteBufferUtils.toHexString(bytes, bytes.length);
+        }
+
+        return ByteBufferUtils.toHexString(buffer.array(),
+                                        buffer.arrayOffset() + buffer.position(),
+                                        buffer.remaining());
+    }
+
+    public static int readFully(InputStream in, byte[] bytes, int length) throws IOException
+    {
+        if (length < 0)
+        {
+            throw new IndexOutOfBoundsException();
+        }
+
+        int total = 0;
+        while (total < length)
+        {
+            int count = in.read(bytes, total, length - total);
+            if (count < 0)
+            {
+                break;
+            }
+            total += count;
+        }
+
+        return total;
+    }
+
+    // Changes buffer position
+    public static ByteBuffer readBytesWithShortLength(ByteBuffer buffer)
+    {
+        return readBytes(buffer, readShortLength(buffer));
+    }
+
+    // Changes buffer position
+    static void writeShortLength(ByteBuffer buffer, int length)
+    {
+        buffer.put((byte) ((length >> 8) & 0xFF));
+        buffer.put((byte) (length & 0xFF));
+    }
+
+    // Doesn't change buffer position
+    static int peekShortLength(ByteBuffer buffer, int position)
+    {
+        int length = (buffer.get(position) & 0xFF) << 8;
+        return length | (buffer.get(position + 1) & 0xFF);
+    }
+
+    // Changes buffer position
+    static int readShortLength(ByteBuffer buffer)
+    {
+        int length = (buffer.get() & 0xFF) << 8;
+        return length | (buffer.get() & 0xFF);
+    }
+
+    // Changes buffer position
+    @SuppressWarnings("RedundantCast")
+    public static ByteBuffer readBytes(ByteBuffer buffer, int length)
+    {
+        ByteBuffer copy = buffer.duplicate();
+        ((Buffer) copy).limit(copy.position() + length);
+        ((Buffer) buffer).position(buffer.position() + length);
+        return copy;
+    }
+
+    public static void skipFully(InputStream is, long length) throws IOException
+    {
+        long skipped = is.skip(length);
+        if (skipped != length)
+        {
+            throw new EOFException("EOF after " + skipped + " bytes out of " + length);
+        }
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/ColumnTypes.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/ColumnTypes.java
new file mode 100644
index 0000000..77134c9
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/ColumnTypes.java
@@ -0,0 +1,130 @@
+/*
+ * 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.cassandra.spark.utils;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.cassandra.spark.data.CqlField;
+
+public final class ColumnTypes
+{
+    private static final int STATIC_MARKER = 0xFFFF;
+
+    private ColumnTypes()
+    {
+        throw new IllegalStateException(getClass() + " is static utility class and shall not be instantiated");
+    }
+
+    public static ByteBuffer buildPartitionKey(List<CqlField> partitionKeys, Object... values)
+    {
+        if (partitionKeys.size() == 1)
+        {
+            // Only 1 partition key
+            CqlField key = partitionKeys.get(0);
+            return key.serialize(values[key.position()]);
+        }
+        else
+        {
+            // Composite partition key
+            ByteBuffer[] buffers = partitionKeys.stream()
+                                                .map(key -> key.serialize(values[key.position()]))
+                                                .toArray(ByteBuffer[]::new);
+            return ColumnTypes.build(false, buffers);
+        }
+    }
+
+    public static ByteBuffer build(boolean isStatic, ByteBuffer... buffers)
+    {
+        int totalLength = isStatic ? 2 : 0;
+        for (ByteBuffer buffer : buffers)
+        {
+            // 2 bytes short length + data length + 1 byte for end-of-component marker
+            totalLength += 2 + buffer.remaining() + 1;
+        }
+
+        ByteBuffer out = ByteBuffer.allocate(totalLength);
+        if (isStatic)
+        {
+            out.putShort((short) STATIC_MARKER);
+        }
+
+        for (ByteBuffer buffer : buffers)
+        {
+            ByteBufferUtils.writeShortLength(out, buffer.remaining());  // Short length
+            out.put(buffer.duplicate());  // Data
+            out.put((byte) 0);  // End-of-component marker
+        }
+        out.flip();
+        return out;
+    }
+
+    // Extract component position from buffer; return null if there are not enough components
+    public static ByteBuffer extractComponent(ByteBuffer buffer, int position)
+    {
+        buffer = buffer.duplicate();
+        readStatic(buffer);
+        int index = 0;
+        while (buffer.remaining() > 0)
+        {
+            ByteBuffer c = ByteBufferUtils.readBytesWithShortLength(buffer);
+            if (index == position)
+            {
+                return c;
+            }
+
+            buffer.get();  // Skip end-of-component
+            ++index;
+        }
+        return null;
+    }
+
+    public static ByteBuffer[] split(ByteBuffer name, int numKeys)
+    {
+        // Assume all components, we'll trunk the array afterwards if need be, but most names will be complete
+        ByteBuffer[] l = new ByteBuffer[numKeys];
+        ByteBuffer buffer = name.duplicate();
+        ColumnTypes.readStatic(buffer);
+        int index = 0;
+        while (buffer.remaining() > 0)
+        {
+            l[index++] = ByteBufferUtils.readBytesWithShortLength(buffer);
+            buffer.get();  // Skip end-of-component
+        }
+        return index == l.length ? l : Arrays.copyOfRange(l, 0, index);
+    }
+
+    public static void readStatic(ByteBuffer buffer)
+    {
+        if (buffer.remaining() < 2)
+        {
+            return;
+        }
+
+        int header = ByteBufferUtils.peekShortLength(buffer, buffer.position());
+        if ((header & 0xFFFF) != STATIC_MARKER)
+        {
+            return;
+        }
+
+        ByteBufferUtils.readShortLength(buffer);  // Skip header
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/ComparisonUtils.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/ComparisonUtils.java
new file mode 100644
index 0000000..f860d25
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/ComparisonUtils.java
@@ -0,0 +1,203 @@
+/*
+ * 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.cassandra.spark.utils;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.net.Inet4Address;
+import java.net.InetAddress;
+import java.nio.ByteBuffer;
+import java.sql.Date;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Map;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.stream.IntStream;
+
+import com.google.common.primitives.UnsignedBytes;
+
+public final class ComparisonUtils
+{
+    private static final Comparator<Object> INET_COMPARATOR = (first, second) ->
+            UnsignedBytes.lexicographicalComparator().compare(((Inet4Address) first).getAddress(),
+                                                              ((Inet4Address) second).getAddress());
+
+    private static final Comparator<Object> NESTED_COMPARATOR = new Comparator<Object>()
+    {
+        @SuppressWarnings({"unchecked", "rawtypes"})
+        public int compare(Object first, Object second)
+        {
+            if (first instanceof Comparable && second instanceof Comparable)
+            {
+                return ((Comparable) first).compareTo(second);
+            }
+            else if (first instanceof Object[] && second instanceof Object[])
+            {
+                Object[] array1 = (Object[]) first;
+                Object[] array2 = (Object[]) second;
+                int position = 0;
+                while (position < array1.length && position < array2.length)
+                {
+                    int comparison = NESTED_COMPARATOR.compare(array1[position], array2[position]);
+                    if (comparison != 0)
+                    {
+                        return comparison;
+                    }
+                    position++;
+                }
+                return Integer.compare(array1.length, array2.length);
+            }
+            else if (first instanceof Map && second instanceof Map)
+            {
+                Map<?, ?> map1 = (Map<?, ?>) first;
+                Map<?, ?> map2 = (Map<?, ?>) second;
+                for (Object key : map1.keySet())
+                {
+                    int comparison = NESTED_COMPARATOR.compare(map1.get(key), map2.get(key));
+                    if (comparison != 0)
+                    {
+                        return comparison;
+                    }
+                }
+                return Integer.compare(map1.size(), map2.size());
+            }
+            else if (first instanceof Collection && second instanceof Collection)
+            {
+                return NESTED_COMPARATOR.compare(((Collection) first).toArray(new Object[0]), ((Collection) second).toArray(new Object[0]));
+            }
+            else if (first instanceof Inet4Address && second instanceof Inet4Address)
+            {
+                return INET_COMPARATOR.compare(first, second);
+            }
+            throw new IllegalStateException("Unexpected comparable type: " + first.getClass().getName());
+        }
+    };
+
+    private ComparisonUtils()
+    {
+        throw new IllegalStateException(getClass() + " is static utility class and shall not be instantiated");
+    }
+
+    public static boolean equals(Object[] first, Object[] second)
+    {
+        if (first == second)
+        {
+            return true;
+        }
+        if (first == null || second == null)
+        {
+            return false;
+        }
+
+        int length = first.length;
+        if (second.length != length)
+        {
+            return false;
+        }
+
+        for (int index = 0; index < length; index++)
+        {
+            if (!equals(first[index], second[index]))
+            {
+                return false;
+            }
+        }
+
+        return true;
+    }
+
+    @SuppressWarnings("unchecked")
+    public static boolean equals(Object first, Object second)
+    {
+        if (first instanceof UUID && second instanceof String)  // Spark does not support UUID. We compare the string values.
+        {
+            return first.toString().equals(second);
+        }
+        else if (first instanceof ByteBuffer && second instanceof byte[])
+        {
+            return Arrays.equals(ByteBufferUtils.getArray((ByteBuffer) first), (byte[]) second);
+        }
+        else if (first instanceof InetAddress && second instanceof byte[])
+        {
+            return Arrays.equals(((InetAddress) first).getAddress(), (byte[]) second);
+        }
+        else if (first instanceof BigInteger && second instanceof BigDecimal)
+        {
+            // Compare the string values
+            return first.toString().equals(second.toString());
+        }
+        else if (first instanceof Integer && second instanceof Date)
+        {
+            return first.equals((int) ((Date) second).toLocalDate().toEpochDay());
+        }
+
+        if (Objects.equals(first, second))
+        {
+            return true;
+        }
+
+        if (first instanceof BigDecimal && second instanceof BigDecimal)
+        {
+            return ((BigDecimal) first).compareTo((BigDecimal) second) == 0;
+        }
+        else if (first instanceof Collection && second instanceof Collection)
+        {
+            Object[] firstArray = ((Collection<Object>) first).toArray();
+            Arrays.sort(firstArray, NESTED_COMPARATOR);
+            Object[] secondArray = ((Collection<Object>) second).toArray();
+            Arrays.sort(secondArray, NESTED_COMPARATOR);
+            return equals(firstArray, secondArray);
+        }
+        else if (first instanceof Map && second instanceof Map)
+        {
+            Object[] firstKeys = ((Map<Object, Object>) first).keySet().toArray();
+            Object[] secondKeys = ((Map<Object, Object>) second).keySet().toArray();
+            if (firstKeys[0] instanceof Comparable)
+            {
+                Arrays.sort(firstKeys);
+                Arrays.sort(secondKeys);
+            }
+            else if (firstKeys[0] instanceof Inet4Address)  // Inet4Address is not Comparable so do byte ordering
+            {
+                Arrays.sort(firstKeys, INET_COMPARATOR);
+                Arrays.sort(secondKeys, INET_COMPARATOR);
+            }
+            if (equals(firstKeys, secondKeys))
+            {
+                Object[] firstValues = new Object[firstKeys.length];
+                Object[] secondValues = new Object[secondKeys.length];
+                IntStream.range(0, firstKeys.length).forEach(position -> {
+                    firstValues[position] = ((Map<Object, Object>) first).get(firstKeys[position]);
+                    secondValues[position] = ((Map<Object, Object>) second).get(secondKeys[position]);
+                });
+                return equals(firstValues, secondValues);
+            }
+            return false;
+        }
+        else if (first instanceof Object[] && second instanceof Object[])
+        {
+            return equals((Object[]) first, (Object[]) second);
+        }
+
+        return false;
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/FutureUtils.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/FutureUtils.java
new file mode 100644
index 0000000..785805e
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/FutureUtils.java
@@ -0,0 +1,165 @@
+/*
+ * 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.cassandra.spark.utils;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.function.Consumer;
+
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+public final class FutureUtils
+{
+    public static final class FutureResult<T>
+    {
+        @Nullable
+        public final T value;
+        @Nullable
+        public final Throwable throwable;
+
+        private FutureResult(@Nullable T value, @Nullable Throwable throwable)
+        {
+            this.value = value;
+            this.throwable = throwable;
+        }
+
+        public static <T> FutureResult<T> failed(@NotNull Throwable throwable)
+        {
+            return new FutureResult<>(null, throwable);
+        }
+
+        public static <T> FutureResult<T> success(@Nullable T value)
+        {
+            return new FutureResult<>(value, null);
+        }
+
+        @Nullable
+        public T value()
+        {
+            return value;
+        }
+
+        public boolean isSuccess()
+        {
+            return throwable == null;
+        }
+    }
+
+    private FutureUtils()
+    {
+        throw new IllegalStateException(getClass() + " is static utility class and shall not be instantiated");
+    }
+
+    /**
+     * Await all futures and combine into single result
+     *
+     * @param <T>                 result type returned by this method
+     * @param futures             list of futures
+     * @param acceptPartialResult if false, fail the entire request if a single failure occurs, if true just log partial failures
+     * @param logger              consumer to log errors
+     * @return result of all combined futures
+     */
+    public static <T> List<T> awaitAll(List<CompletableFuture<T>> futures,
+                                       boolean acceptPartialResult,
+                                       Consumer<Throwable> logger)
+    {
+        List<T> result = new ArrayList<>(futures.size() * 10);  // TODO: Comment on why allocate tenfold
+        for (CompletableFuture<T> future : futures)
+        {
+            FutureResult<T> futureResult = await(future, logger);
+            if (futureResult.throwable != null)
+            {
+                // Failed
+                if (!acceptPartialResult)
+                {
+                    throw new RuntimeException(ThrowableUtils.rootCause(futureResult.throwable));
+                }
+            }
+            else if (futureResult.value != null)
+            {
+                // Success
+                result.add(futureResult.value);
+            }
+        }
+        return result;
+    }
+
+    /**
+     * Await a future and return result
+     *
+     * @param future the future
+     * @param logger consumer to log errors
+     * @param <T>    result type returned by this method
+     * @return result of the future
+     */
+    @NotNull
+    public static <T> FutureResult<T> await(CompletableFuture<T> future, Consumer<Throwable> logger)
+    {
+        try
+        {
+            return FutureResult.success(future.get());
+        }
+        catch (InterruptedException exception)
+        {
+            Thread.currentThread().interrupt();
+            throw new RuntimeException(exception);
+        }
+        catch (ExecutionException exception)
+        {
+            logger.accept(exception);
+            return FutureResult.failed(exception);
+        }
+    }
+
+    /**
+     * Combine futures into a single future that completes when all futures complete successfully, or fails if any future fails
+     *
+     * @param <T>     result type returned by this method
+     * @param futures array of futures
+     * @return a single future that combines all future results
+     */
+    public static <T> CompletableFuture<List<T>> combine(List<CompletableFuture<T>> futures)
+    {
+        CompletableFuture<List<T>> result = new CompletableFuture<>();
+        CompletableFuture.allOf(futures.toArray(new CompletableFuture[0]))
+                         .whenComplete((aVoid, aThrowable) -> {
+                             if (aThrowable == null)
+                             {
+                                 try
+                                 {
+                                     // Combine future results into a single list: get is called, but it will not block as all the futures have completed
+                                     result.complete(awaitAll(futures, false, throwable -> { }));
+                                 }
+                                 catch (Throwable throwable)
+                                 {
+                                     result.completeExceptionally(ThrowableUtils.rootCause(throwable));
+                                 }
+                             }
+                             else
+                             {
+                                 result.completeExceptionally(aThrowable);
+                             }
+                         });
+        return result;
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/IOUtils.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/IOUtils.java
new file mode 100644
index 0000000..687141d
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/IOUtils.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.cassandra.spark.utils;
+
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Comparator;
+import java.util.function.Consumer;
+import java.util.stream.Stream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.jetbrains.annotations.Nullable;
+
+public final class IOUtils
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(IOUtils.class);
+
+    public static final int DEFAULT_CDC_BUFFER_SIZE = 4096;
+
+    private IOUtils()
+    {
+        throw new IllegalStateException(getClass() + " is static utility class and shall not be instantiated");
+    }
+
+    public static void closeQuietly(@Nullable AutoCloseable closeable)
+    {
+        if (closeable != null)
+        {
+            try
+            {
+                closeable.close();
+            }
+            catch (Throwable throwable)
+            {
+                LOGGER.warn("Exception closing {}", closeable.getClass().getName(), throwable);
+            }
+        }
+    }
+
+    public static boolean isNotEmpty(Path path)
+    {
+        return size(path) > 0;
+    }
+
+    public static long size(Path path)
+    {
+        try
+        {
+            return Files.size(path);
+        }
+        catch (IOException exception)
+        {
+            throw new RuntimeException(exception);
+        }
+    }
+
+    public static void clearDirectory(Path path)
+    {
+        clearDirectory(path, log -> { });
+    }
+
+    public static void clearDirectory(Path path, Consumer<Path> logger)
+    {
+        try (Stream<Path> walker = Files.walk(path))
+        {
+            walker.sorted(Comparator.reverseOrder())
+                  .filter(Files::isRegularFile)
+                  .forEach(file -> {
+                      try
+                      {
+                          logger.accept(file);
+                          Files.delete(file);
+                      }
+                      catch (IOException exception)
+                      {
+                          throw new RuntimeException(exception);
+                      }
+                  });
+        }
+        catch (IOException exception)
+        {
+            throw new RuntimeException(exception);
+        }
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/LoggerHelper.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/LoggerHelper.java
new file mode 100644
index 0000000..63467ed
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/LoggerHelper.java
@@ -0,0 +1,152 @@
+/*
+ * 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.cassandra.spark.utils;
+
+import java.util.Arrays;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Helper class to automatically append fixed logging values for every message
+ */
+public final class LoggerHelper
+{
+    private final Logger logger;
+    private final String keys;
+    private final Object[] fixedArgumentss;
+
+    public LoggerHelper(Logger logger, Object... fixedArguments)
+    {
+        this.logger = logger;
+        this.keys = buildKey(fixedArguments);
+        this.fixedArgumentss = extractArguments(fixedArguments);
+    }
+
+    private static String buildKey(Object... args)
+    {
+        Preconditions.checkArgument(args.length % 2 == 0, "Expect even number of key/value pairs in fixedArgs");
+        return " " + IntStream.range(0, args.length / 2)
+                              .map(index -> index * 2)
+                              .mapToObj(index -> args[index])
+                              .map(key -> key + "={}")
+                              .collect(Collectors.joining(" "));
+    }
+
+    private static Object[] extractArguments(Object... arguments)
+    {
+        Preconditions.checkArgument(arguments.length % 2 == 0, "Expect even number of key/value pairs in fixedArgs");
+        return IntStream.range(0, arguments.length / 2)
+                        .map(index -> index * 2 + 1)
+                        .mapToObj(index -> arguments[index])
+                        .toArray(Object[]::new);
+    }
+
+    public void trace(String message, Object... arguments)
+    {
+        if (logger.isTraceEnabled())
+        {
+            logger.trace(logMsg(message, arguments), buildArguments(arguments));
+        }
+    }
+
+    @SafeVarargs
+    public final void trace(String message, Supplier<Object>... arguments)
+    {
+        if (logger.isTraceEnabled())
+        {
+            Object[] evaluatedArguments = Arrays.stream(arguments)
+                                                .map(Supplier::get)
+                                                .toArray();
+            logger.trace(logMsg(message, evaluatedArguments), buildArguments(evaluatedArguments));
+        }
+    }
+
+    public void debug(String message, Object... arguments)
+    {
+        if (logger.isDebugEnabled())
+        {
+            logger.debug(logMsg(message, arguments), buildArguments(arguments));
+        }
+    }
+
+    @SafeVarargs
+    public final void debug(String message, Supplier<Object>... arguments)
+    {
+        if (logger.isDebugEnabled())
+        {
+            Object[] evaluatedArguments = Arrays.stream(arguments)
+                                                .map(Supplier::get)
+                                                .toArray();
+            logger.debug(logMsg(message, evaluatedArguments), buildArguments(evaluatedArguments));
+        }
+    }
+
+    public void info(String message, Object... arguments)
+    {
+        logger.info(logMsg(message, arguments), buildArguments(arguments));
+    }
+
+    public void warn(String message, Throwable throwable, Object... arguments)
+    {
+        logger.warn(logMsg(message, arguments), buildArguments(throwable, arguments));
+    }
+
+    public void error(String message, Throwable throwable, Object... arguments)
+    {
+        logger.error(logMsg(message, arguments), buildArguments(throwable, arguments));
+    }
+
+    @VisibleForTesting
+    String logMsg(String message, Object... arguments)
+    {
+        message += keys;
+        if (0 < arguments.length)
+        {
+            message += buildKey(arguments);
+        }
+        return message;
+    }
+
+    private Object[] buildArguments(Object... arguments)
+    {
+        return buildArguments(null, arguments);
+    }
+
+    @VisibleForTesting
+    Object[] buildArguments(@Nullable Throwable throwable, Object... arguments)
+    {
+        Object[] variableArguments = extractArguments(arguments);
+        Object[] allArguments = new Object[variableArguments.length + fixedArgumentss.length + (throwable != null ? 1 : 0)];
+        System.arraycopy(fixedArgumentss, 0, allArguments, 0, fixedArgumentss.length);
+        System.arraycopy(variableArguments, 0, allArguments, fixedArgumentss.length, variableArguments.length);
+        if (throwable != null)
+        {
+            allArguments[variableArguments.length + fixedArgumentss.length] = throwable;
+        }
+        return allArguments;
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/MapUtils.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/MapUtils.java
new file mode 100644
index 0000000..ee95c50
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/MapUtils.java
@@ -0,0 +1,213 @@
+/*
+ * 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.cassandra.spark.utils;
+
+import java.util.Locale;
+import java.util.Map;
+import java.util.function.Supplier;
+
+/**
+ * Utility methods for {@link Map}
+ */
+public final class MapUtils
+{
+    private MapUtils()
+    {
+        throw new IllegalStateException(getClass() + " is static utility class and shall not be instantiated");
+    }
+
+    /**
+     * Returns the lower-cased key using {@link Locale#ROOT}
+     *
+     * @param key the key
+     * @return the lower-cased key using {@link Locale#ROOT}
+     */
+    public static String lowerCaseKey(String key)
+    {
+        return key != null ? key.toLowerCase(Locale.ROOT) : null;
+    }
+
+    public static String getOrThrow(Map<String, String> options, String key)
+    {
+        return getOrThrow(options, key, key);
+    }
+
+    public static String getOrThrow(Map<String, String> options, String key, String name)
+    {
+        return getOrThrow(options, key, throwable(name));
+    }
+
+    public static Supplier<RuntimeException> throwable(String name)
+    {
+        return () -> new RuntimeException(String.format("No %s specified", name));
+    }
+
+    public static String getOrThrow(Map<String, String> options, String key, Supplier<RuntimeException> throwable)
+    {
+        return getOrHandle(options, key, () -> {
+            throw throwable.get();
+        });
+    }
+
+    public static String getOrHandle(Map<String, String> options, String key, Supplier<String> handler)
+    {
+        String value = options.get(lowerCaseKey(key));
+        if (value == null)
+        {
+            return handler.get();
+        }
+        return value;
+    }
+
+    /**
+     * Returns the boolean value for the given {@code key} in the {@code options} map. The key is lower-cased before
+     * accessing the map. The {@code defaultValue} is returned when the key doesn't match any element from the map.
+     *
+     * @param options      the map
+     * @param key          the key to the map
+     * @param defaultValue the default value
+     * @return the boolean value
+     */
+    public static boolean getBoolean(Map<String, String> options, String key, boolean defaultValue)
+    {
+        return getBoolean(options, key, defaultValue, null);
+    }
+
+    /**
+     * Returns the boolean value for the given {@code key} in the {@code options} map. The key is lower-cased before
+     * accessing the map. The {@code defaultValue} is returned when the key doesn't match any element from the map.
+     *
+     * @param options      the map
+     * @param key          the key to the map
+     * @param defaultValue the default value
+     * @param displayName  an optional name to display in the error message
+     * @return the boolean value
+     */
+    public static boolean getBoolean(Map<String, String> options, String key, boolean defaultValue, String displayName)
+    {
+        String value = options.get(lowerCaseKey(key));
+        // We can't use `Boolean.parseBoolean` here, as it returns false for invalid strings
+        if (value == null)
+        {
+            return defaultValue;
+        }
+        else if (value.equalsIgnoreCase("true"))
+        {
+            return true;
+        }
+        else if (value.equalsIgnoreCase("false"))
+        {
+            return false;
+        }
+        displayName = displayName != null ? displayName : key;
+        throw new IllegalArgumentException("Key " + displayName + " with value " + value + " is not a valid boolean string");
+    }
+
+    /**
+     * Returns the int value for the given {@code key} in the {@code options} map. The key is lower-cased before
+     * accessing the map. The {@code defaultValue} is returned when the key doesn't match any element from the map.
+     *
+     * @param options      the map
+     * @param key          the key to the map
+     * @param defaultValue the default value
+     * @return the int value
+     */
+    public static int getInt(Map<String, String> options, String key, int defaultValue)
+    {
+        return getInt(options, key, defaultValue, null);
+    }
+
+    /**
+     * Returns the int value for the given {@code key} in the {@code options} map. The key is lower-cased before
+     * accessing the map. The {@code defaultValue} is returned when the key doesn't match any element from the map.
+     *
+     * @param options      the map
+     * @param key          the key to the map
+     * @param defaultValue the default value
+     * @param displayName  an optional name to display in the error message
+     * @return the int value
+     */
+    public static int getInt(Map<String, String> options, String key, int defaultValue, String displayName)
+    {
+        String value = options.get(lowerCaseKey(key));
+        try
+        {
+            return value != null ? Integer.parseInt(value) : defaultValue;
+        }
+        catch (NumberFormatException exception)
+        {
+            displayName = displayName != null ? displayName : key;
+            throw new IllegalArgumentException("Key " + displayName + " with value " + value + " is not a valid integer string.",
+                                               exception);
+        }
+    }
+
+    /**
+     * Returns the long value for the given {@code key} in the {@code options} map. The key is lower-cased before
+     * accessing the map. The {@code defaultValue} is returned when the key doesn't match any element from the map.
+     *
+     * @param options      the map
+     * @param key          the key to the map
+     * @param defaultValue the default value
+     * @return the long value
+     */
+    public static long getLong(Map<String, String> options, String key, long defaultValue)
+    {
+        String value = options.get(lowerCaseKey(key));
+        return value != null ? Long.parseLong(value) : defaultValue;
+    }
+
+    public static <T extends Enum<T>> T getEnumOption(Map<String, String> options, String key, T defaultValue)
+    {
+        return getEnumOption(options, key, defaultValue, null);
+    }
+
+    @SuppressWarnings("unchecked")
+    public static <T extends Enum<T>> T getEnumOption(Map<String, String> options,
+                                                      String key, T defaultValue,
+                                                      String displayName)
+    {
+        String value = options.get(lowerCaseKey(key));
+        try
+        {
+            return value != null ? (T) Enum.valueOf(defaultValue.getClass(), value) : defaultValue;
+        }
+        catch (IllegalArgumentException exception)
+        {
+            displayName = displayName != null ? displayName : key;
+            throw new IllegalArgumentException("Key " + displayName + " with value " + value + " is not a valid Enum of type " + defaultValue.getClass() + ".",
+                                               exception);
+        }
+    }
+
+    /**
+     * Returns the String value for the given {@code key} in the {@code options} map. The key is lower-cased before
+     * accessing the map. The {@code defaultValue} is returned when the key doesn't match any element from the map.
+     *
+     * @param options      the map
+     * @param key          the key to the map
+     * @param defaultValue the default value
+     * @return the long value
+     */
+    public static String getOrDefault(Map<String, String> options, String key, String defaultValue)
+    {
+        return options.getOrDefault(lowerCaseKey(key), defaultValue);
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/RandomUtils.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/RandomUtils.java
new file mode 100644
index 0000000..8fd8d02
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/RandomUtils.java
@@ -0,0 +1,107 @@
+/*
+ * 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.cassandra.spark.utils;
+
+import java.math.BigInteger;
+import java.net.InetAddress;
+import java.nio.ByteBuffer;
+import java.util.Random;
+import java.util.UUID;
+
+import com.google.common.net.InetAddresses;
+
+import org.apache.cassandra.spark.data.CqlField;
+import org.apache.cassandra.spark.data.partitioner.Partitioner;
+
+public final class RandomUtils
+{
+    private static final int MIN_COLLECTION_SIZE = 16;
+
+    public static final Random RANDOM = new Random();
+
+    private RandomUtils()
+    {
+        throw new IllegalStateException(getClass() + " is static utility class and shall not be instantiated");
+    }
+
+    public static byte randomByte()
+    {
+        return randomBytes(1)[0];
+    }
+
+    public static byte[] randomBytes(int size)
+    {
+        byte[] bytes = new byte[size];
+        RANDOM.nextBytes(bytes);
+        return bytes;
+    }
+
+    public static ByteBuffer randomByteBuffer(int length)
+    {
+        return ByteBuffer.wrap(randomBytes(length));
+    }
+
+    public static int randomPositiveInt(int bound)
+    {
+        return RANDOM.nextInt(bound - 1) + 1;
+    }
+
+    public static BigInteger randomBigInteger(Partitioner partitioner)
+    {
+        BigInteger range = partitioner.maxToken().subtract(partitioner.minToken());
+        int length = partitioner.maxToken().bitLength();
+        BigInteger result = new BigInteger(length, RandomUtils.RANDOM);
+        if (result.compareTo(partitioner.minToken()) < 0)
+        {
+            result = result.add(partitioner.minToken());
+        }
+        if (result.compareTo(range) >= 0)
+        {
+            result = result.mod(range).add(partitioner.minToken());
+        }
+        return result;
+    }
+
+    /**
+     * Returns a random Type 1 (time-based) UUID.
+     *
+     * Since Java does not natively support creation of Type 1 (time-based) UUIDs, and in order to avoid introducing
+     * a dependency on {@code org.apache.cassandra.utils.UUIDGen}, we obtain a Type 4 (random) UUID and "fix" it.
+     *
+     * @return a random Type 1 (time-based) UUID
+     */
+    public static UUID getRandomTimeUUIDForTesting()
+    {
+        UUID uuid = UUID.randomUUID();
+        return new UUID(uuid.getMostSignificantBits()  ^ 0x0000000000005000L,   // Change UUID version from 4 to 1
+                        uuid.getLeastSignificantBits() | 0x0000010000000000L);  // Always set multicast bit to 1
+    }
+
+    @SuppressWarnings("UnstableApiUsage")
+    public static InetAddress randomInet()
+    {
+        return InetAddresses.fromInteger(RANDOM.nextInt());
+    }
+
+    public static Object randomValue(CqlField.CqlType type)
+    {
+        return type.randomValue(MIN_COLLECTION_SIZE);
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/SparkClassLoaderOverride.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/SparkClassLoaderOverride.java
new file mode 100644
index 0000000..7824412
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/SparkClassLoaderOverride.java
@@ -0,0 +1,85 @@
+/*
+ * 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.cassandra.spark.utils;
+
+import java.io.ObjectInputStream;
+import java.lang.reflect.Field;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.spark.data.CqlTable;
+import org.apache.spark.serializer.JavaDeserializationStream;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * This class is an {@link AutoCloseable} wrapper that allows to temporarily substitute the instance of
+ * {@link ClassLoader} in use by a {@link ObjectInputStream} constructed by Spark for performing JDK deserialization.
+ * Such substitution is required in order to resolve types of Cassandra-version-dependent objects,
+ * specifically those defined under {@code org.apache.cassandra.spark.data.types} and used by the {@link CqlTable}.
+ */
+public class SparkClassLoaderOverride implements AutoCloseable
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(SparkClassLoaderOverride.class);
+
+    @NotNull  private final ObjectInputStream in;
+    @Nullable private final ClassLoader loader;
+
+    public SparkClassLoaderOverride(@NotNull ObjectInputStream in, @NotNull ClassLoader loader)
+    {
+        this.in = in;
+        this.loader = swapClassLoader(in, loader);
+    }
+
+    @Nullable
+    private static ClassLoader swapClassLoader(@NotNull ObjectInputStream in, @NotNull ClassLoader loader)
+    {
+        try
+        {
+            Field outerField = in.getClass().getDeclaredField("$outer");
+            outerField.setAccessible(true);
+            JavaDeserializationStream outerValue = (JavaDeserializationStream) outerField.get(in);
+            Field loaderField = outerValue.getClass().getDeclaredField("org$apache$spark$serializer$JavaDeserializationStream$$loader");
+            loaderField.setAccessible(true);
+            ClassLoader loaderValue = (ClassLoader) loaderField.get(outerValue);
+            loaderField.set(outerValue, loader);
+            return loaderValue;
+        }
+        catch (NullPointerException | NoSuchFieldException | IllegalAccessException | ClassCastException exception)
+        {
+            // Ignore all of the above exceptions: if any of them occurs, we are either not deserializing
+            // from a Spark input stream, or Spark input stream's internal implementation has changed -
+            // in either case we should abort the substitution and let deserialization fail
+            // when resolving types of Cassandra-version-specific objects
+            LOGGER.warn("Cannot override class loader in a Spark object input stream", exception);
+            return null;
+        }
+    }
+
+    @Override
+    public void close()
+    {
+        if (loader != null)
+        {
+            swapClassLoader(in, loader);
+        }
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/TemporaryDirectory.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/TemporaryDirectory.java
new file mode 100644
index 0000000..fc6bf21
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/TemporaryDirectory.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.cassandra.spark.utils;
+
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.UUID;
+
+import org.apache.commons.io.FileUtils;
+
+public class TemporaryDirectory implements AutoCloseable
+{
+    private final Path directory;
+
+    public TemporaryDirectory() throws IOException
+    {
+        directory = Files.createTempDirectory(UUID.randomUUID().toString());
+    }
+
+    public Path path()
+    {
+        return directory;
+    }
+
+    @Override
+    public void close() throws IOException
+    {
+        FileUtils.deleteDirectory(directory.toFile());
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/ThrowableUtils.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/ThrowableUtils.java
new file mode 100644
index 0000000..8e2fc36
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/ThrowableUtils.java
@@ -0,0 +1,70 @@
+/*
+ * 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.cassandra.spark.utils;
+
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+public final class ThrowableUtils
+{
+    private ThrowableUtils()
+    {
+        throw new IllegalStateException(getClass() + " is static utility class and shall not be instantiated");
+    }
+
+    /**
+     * Find root cause of throwable or this throwable if no prior cause
+     *
+     * @param throwable throwable
+     * @return initial cause throwable
+     */
+    @NotNull
+    public static Throwable rootCause(@NotNull Throwable throwable)
+    {
+        while (throwable.getCause() != null)
+        {
+            throwable = throwable.getCause();
+        }
+        return throwable;
+    }
+
+    /**
+     * Find first throwable of type matching ofType parameter or null if not exists
+     *
+     * @param <T>       generic type of expected return value
+     * @param throwable throwable
+     * @param ofType    type of class expected
+     * @return first throwable of type matching parameter ofType or null if cannot be found
+     */
+    @Nullable
+    public static <T extends Throwable> T rootCause(@NotNull Throwable throwable, @NotNull Class<T> ofType)
+    {
+        while (throwable.getCause() != null)
+        {
+            if (ofType.isInstance(throwable))
+            {
+                return ofType.cast(throwable);
+            }
+            throwable = throwable.getCause();
+        }
+
+        return ofType.isInstance(throwable) ? ofType.cast(throwable) : null;
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/Throwing.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/Throwing.java
new file mode 100644
index 0000000..f4a5e4e
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/Throwing.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.cassandra.spark.utils;
+
+public final class Throwing
+{
+    private Throwing()
+    {
+        throw new IllegalStateException(getClass() + " is static utility class and shall not be instantiated");
+    }
+
+    @FunctionalInterface
+    public interface Consumer<T>
+    {
+        void accept(T object) throws Exception;
+    }
+
+    public static <T> java.util.function.Consumer<T> consumer(Consumer<T> consumer)
+    {
+        return object -> {
+            try
+            {
+                consumer.accept(object);
+            }
+            catch (Exception exception)
+            {
+                throw new RuntimeException(exception);
+            }
+        };
+    }
+
+    @FunctionalInterface
+    public interface Function<T, R>
+    {
+        R apply(T object) throws Exception;
+    }
+
+    public static <T, R> java.util.function.Function<T, R> function(Function<T, R> function)
+    {
+        return object -> {
+            try
+            {
+                return function.apply(object);
+            }
+            catch (Exception exception)
+            {
+                throw new RuntimeException(exception);
+            }
+        };
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/TimeProvider.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/TimeProvider.java
new file mode 100644
index 0000000..950cb6a
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/TimeProvider.java
@@ -0,0 +1,34 @@
+/*
+ * 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.cassandra.spark.utils;
+
+/**
+ * Provides current time
+  */
+@FunctionalInterface
+public interface TimeProvider
+{
+    TimeProvider INSTANCE = () -> (int) Math.floorDiv(System.currentTimeMillis(), 1000L);
+
+    /**
+     * @return current time in truncated seconds
+     */
+    int nowInTruncatedSeconds();
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/streaming/SSTableInputStream.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/streaming/SSTableInputStream.java
new file mode 100644
index 0000000..65f90db
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/streaming/SSTableInputStream.java
@@ -0,0 +1,607 @@
+/*
+ * 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.cassandra.spark.utils.streaming;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.time.Duration;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.cassandra.spark.data.SSTable;
+import org.apache.cassandra.spark.stats.Stats;
+import org.apache.cassandra.spark.utils.ThrowableUtils;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * The InputStream into the CompactionIterator needs to be a blocking {@link java.io.InputStream},
+ * but we don't want to block on network calls, or buffer too much data in memory otherwise we will hit OOMs for large Data.db files.
+ * <p>
+ * This helper class uses the {@link SSTableSource} implementation provided to asynchronously read
+ * the T bytes on-demand, managing flow control if not ready for more bytes and buffering enough without reading entirely into memory.
+ * <p>
+ * The generic {@link SSTableSource} allows users to pass in their own implementations to read from any source.
+ * <p>
+ * This enables the Bulk Reader library to scale to read many SSTables without OOMing, and controls the flow by
+ * buffering more bytes on-demand as the data is drained.
+ * <p>
+ * This class expects the consumer thread to be single-threaded, and the producer thread to be single-threaded OR serialized to ensure ordering of events.
+ *
+ * @param <T> SSTable
+ */
+@SuppressWarnings({"WeakerAccess", "unused"})
+public class SSTableInputStream<T extends SSTable> extends InputStream implements StreamConsumer
+{
+    private static final StreamBuffer.ByteArrayWrapper END_MARKER = StreamBuffer.wrap(new byte[0]);
+    private static final StreamBuffer.ByteArrayWrapper FINISHED_MARKER = StreamBuffer.wrap(new byte[0]);
+    private static final StreamBuffer.ByteArrayWrapper ERROR_MARKER = StreamBuffer.wrap(new byte[0]);
+
+    private enum StreamState
+    {
+        Init,
+        Reading,
+        NextBuffer,
+        End,
+        Closed
+    }
+
+    private final BlockingQueue<StreamBuffer> queue;
+    private final SSTableSource<T> source;
+    private final Stats stats;
+    private final long startTimeNanos;
+
+    // Variables accessed by both producer, consumer & timeout thread so must be volatile or atomic
+    private volatile Throwable throwable = null;
+    private volatile boolean activeRequest = false;
+    private volatile boolean closed = false;
+    private final AtomicLong bytesWritten = new AtomicLong(0L);
+
+    // Variables only used by the InputStream consumer thread so do not need to be volatile or atomic
+    private long rangeStart = 0L;
+    private long bytesRead = 0L;
+    private long timeBlockedNanos = 0L;
+    private boolean skipping = false;
+    private StreamState state = StreamState.Init;
+    private StreamBuffer currentBuffer = null;
+    private int position;
+    private int length;
+
+    /**
+     * @param source SSTableSource to async provide the bytes after {@link SSTableSource#request(long, long, StreamConsumer)} is called
+     *
+     * @param stats {@link Stats} implementation for recording instrumentation
+     */
+    public SSTableInputStream(SSTableSource<T> source, Stats stats)
+    {
+        this.source = source;
+        this.queue = new LinkedBlockingQueue<>();
+        this.startTimeNanos = System.nanoTime();
+        this.stats = stats;
+    }
+
+    public long startTimeNanos()
+    {
+        return startTimeNanos;
+    }
+
+    public long timeBlockedNanos()
+    {
+        return timeBlockedNanos;
+    }
+
+    public long bytesWritten()
+    {
+        return bytesWritten.get();
+    }
+
+    public long bytesRead()
+    {
+        return bytesRead;
+    }
+
+    public long bytesBuffered()
+    {
+        return bytesWritten() - bytesRead();
+    }
+
+    public boolean isFinished()
+    {
+        return bytesWritten() >= source.size();
+    }
+
+    private boolean isClosed()
+    {
+        return state == StreamState.Closed;
+    }
+
+    /**
+     * Can request more bytes if:
+     * 1. a request not already in-flight
+     * 2. not in the middle of skip method call
+     * 3. the queue buffer is not full i.e. bytes in memory not greater than or equal to maxBufferSize
+     * 4. the InputStream is not closed
+     *
+     * @return true if can request more bytes
+     */
+    private boolean canRequestMore()
+    {
+        return !(activeRequest || skipping || isBufferFull() || isClosed());
+    }
+
+    /**
+     * Maybe request more bytes if possible
+     */
+    private void maybeRequestMore()
+    {
+        if (canRequestMore())
+        {
+            requestMore();
+        }
+    }
+
+    /**
+     * Request more bytes using {@link SSTableSource#request(long, long, StreamConsumer)} for the next range
+     */
+    private void requestMore()
+    {
+        if (rangeStart >= source.size())
+        {
+            if (isFinished())
+            {
+                // If user skips to end of stream we still need to complete,
+                // otherwise read() blocks waiting for FINISHED_MARKER
+                queue.add(FINISHED_MARKER);
+            }
+            return;  // Finished
+        }
+
+        long chunkSize = rangeStart == 0 ? source.headerChunkSize() : source.chunkBufferSize();
+        long rangeEnd = Math.min(source.size(), rangeStart + chunkSize);
+        if (rangeEnd >= rangeStart)
+        {
+            activeRequest = true;
+            source.request(rangeStart, rangeEnd, this);
+            rangeStart += chunkSize + 1;  // Increment range start pointer for next request
+        }
+        else
+        {
+            throw new IllegalStateException(String.format("Tried to request invalid range start=%d end=%d",
+                                                          rangeStart, rangeEnd));
+        }
+    }
+
+    /**
+     * The number of bytes buffered is greater than or equal to {@link SSTableSource#maxBufferSize()}
+     * so wait for queue to drain before requesting more
+     *
+     * @return true if queue is full
+     */
+    public boolean isBufferFull()
+    {
+        return bytesBuffered() >= source.maxBufferSize();
+    }
+
+    // Timeout
+
+    /**
+     * @param timeout           duration timeout
+     * @param nowNanos          current time now in nanoseconds
+     * @param lastActivityNanos last activity time in nanoseconds
+     * @return the timeout remaining in nanoseconds, or less than or equal to 0 if timeout already expired
+     */
+    public static long timeoutLeftNanos(Duration timeout, long nowNanos, long lastActivityNanos)
+    {
+        return Math.min(timeout.toNanos(), timeout.toNanos() - (nowNanos - lastActivityNanos));
+    }
+
+    private Throwable timeoutException(Duration timeout)
+    {
+        return new TimeoutException(String.format("No activity on SSTableInputStream for %d seconds",
+                                                  timeout.getSeconds()));
+    }
+
+    private void timeoutError(Duration timeout)
+    {
+        onError(timeoutException(timeout));
+    }
+
+    /**
+     * {@link StreamConsumer} method implementations
+     */
+    @Override
+    public void onRead(StreamBuffer buffer)
+    {
+        int length = buffer.readableBytes();
+        if (length <= 0 || closed)
+        {
+            return;
+        }
+        bytesWritten.addAndGet(length);
+        queue.add(buffer);
+        stats.inputStreamBytesWritten(source, length);
+    }
+
+    @Override
+    public void onEnd()
+    {
+        activeRequest = false;
+        if (isFinished())
+        {
+            queue.add(FINISHED_MARKER);
+        }
+        else
+        {
+            queue.add(END_MARKER);
+        }
+    }
+
+    @Override
+    public void onError(@NotNull Throwable throwable)
+    {
+        this.throwable = ThrowableUtils.rootCause(throwable);
+        activeRequest = false;
+        queue.add(ERROR_MARKER);
+        stats.inputStreamFailure(source, throwable);
+    }
+
+    /**
+     * {@link java.io.InputStream} method implementations
+     */
+    @Override
+    public int available()
+    {
+        return Math.toIntExact(bytesBuffered());
+    }
+
+    @Override
+    public boolean markSupported()
+    {
+        return false;
+    }
+
+    /**
+     * If the schema contains large blobs that can be filtered, then we can more efficiently
+     * skip bytes by incrementing the startRange and avoid wastefully streaming bytes across the network
+     *
+     * @param count number of bytes
+     * @return number of bytes actually skipped
+     * @throws IOException IOException
+     */
+    @Override
+    public long skip(long count) throws IOException
+    {
+        if (count <= 0)
+        {
+            return 0;
+        }
+        else if (count <= bytesBuffered())
+        {
+            long actual = super.skip(count);
+            stats.inputStreamBytesSkipped(source, actual, 0);
+            return actual;
+        }
+
+        skipping = true;
+        long remaining = count;
+        while (activeRequest || !queue.isEmpty())
+        {
+            // Drain any buffered bytes and block until active request completes and the queue is empty
+            remaining -= super.skip(remaining);
+            if (remaining <= 0)
+            {
+                break;
+            }
+        }
+
+        // Increment range start pointer to efficiently skip without reading bytes across the network unnecessarily
+        if (remaining > 0)
+        {
+            rangeStart += remaining;
+            bytesWritten.addAndGet(remaining);
+            bytesRead += remaining;
+        }
+
+        // Remove skip marker and resume requesting bytes
+        skipping = false;
+        switch (state)
+        {
+            case Reading:
+            case NextBuffer:
+                // Stream is active so request more bytes if queue is not full
+                maybeRequestMore();
+                break;
+            default:
+                // If skip() is called before calling read() the Stream will be in StreamState.Init,
+                // in this case we need to initialize the stream before request more bytes
+                checkState();
+        }
+        stats.inputStreamBytesSkipped(source, count - remaining, remaining);
+        return count;
+    }
+
+    /**
+     * Allows directly reading into ByteBuffer without intermediate copy
+     *
+     * @param buffer the ByteBuffer
+     * @throws EOFException if attempts to read beyond the end of the file
+     * @throws IOException  for failure during I/O
+     */
+    public void read(ByteBuffer buffer) throws IOException
+    {
+        for (int remainingLength = buffer.remaining(); 0 < remainingLength; remainingLength = buffer.remaining())
+        {
+            if (checkState() < 0)
+            {
+                throw new EOFException();
+            }
+            int readLength = Math.min(length - position, remainingLength);
+            if (0 < readLength)
+            {
+                currentBuffer.getBytes(position, buffer, readLength);
+                position += readLength;
+                bytesRead += readLength;
+            }
+            maybeReleaseBuffer();
+        }
+    }
+
+    // Copied from JDK11 jdk.internal.util.Preconditions.checkFromIndexSize()
+    private static <X extends RuntimeException> void checkFromIndexSize(int fromIndex, int size, int length)
+    {
+        if ((length | fromIndex | size) < 0 || size > length - fromIndex)
+        {
+            throw new IndexOutOfBoundsException(String.format("Index out of bounds fromIndex=%d, size=%d, length=%d",
+                                                              fromIndex, size, length));
+        }
+    }
+
+    @Override
+    public int read(byte[] buffer, int offset, int length) throws IOException
+    {
+        SSTableInputStream.checkFromIndexSize(offset, length, buffer.length);
+        if (length == 0)
+        {
+            return 0;
+        }
+
+        if (checkState() < 0)
+        {
+            return -1;
+        }
+
+        int readLength = Math.min(this.length - position, length);
+        if (readLength > 0)
+        {
+            currentBuffer.getBytes(position, buffer, offset, readLength);
+            position += readLength;
+            bytesRead += readLength;
+        }
+        maybeReleaseBuffer();
+        return readLength;
+    }
+
+    @Override
+    public int read() throws IOException
+    {
+        do
+        {
+            if (checkState() < 0)
+            {
+                return -1;
+            }
+
+            if (currentBuffer.readableBytes() == 0)
+            {
+                // Current buffer might be empty, normally if it is a marker buffer e.g. END_MARKER
+                maybeReleaseBuffer();
+            }
+        } while (currentBuffer == null);
+
+        // Convert to unsigned byte
+        int unsigned = currentBuffer.getByte(position++) & 0xFF;
+        bytesRead++;
+        maybeReleaseBuffer();
+        return unsigned;
+    }
+
+    @Override
+    public void close()
+    {
+        if (state == StreamState.Closed)
+        {
+            return;
+        }
+        else if (state != StreamState.End)
+        {
+            end();
+        }
+        state = StreamState.Closed;
+        closed = true;
+        releaseBuffer();
+        queue.clear();
+    }
+
+    @Override
+    public void reset() throws IOException
+    {
+        throw new IOException("reset not supported");
+    }
+
+    // Internal Methods for java.io.InputStream
+
+    /**
+     * If position >= length, we have finished with this {@link SSTableInputStream#currentBuffer} so release and
+     * move to the State {@link StreamState#NextBuffer} so next buffer is popped from the {@link LinkedBlockingQueue}
+     * when {@link InputStream#read()} or {@link InputStream#read(byte[], int, int)} is next called
+     */
+    private void maybeReleaseBuffer()
+    {
+        maybeRequestMore();
+        if (position < length)
+        {
+            // Still bytes remaining in the currentBuffer so keep reading
+            return;
+        }
+
+        releaseBuffer();
+        state = StreamState.NextBuffer;
+        stats.inputStreamByteRead(source, position, queue.size(), (int) (position * 100.0 / (double) source.size()));
+    }
+
+    /**
+     * Release current buffer
+     */
+    private void releaseBuffer()
+    {
+        if (currentBuffer != null)
+        {
+            currentBuffer.release();
+            currentBuffer = null;
+        }
+    }
+
+    /**
+     * Pop next buffer from the queue, block on {@link LinkedBlockingQueue} until bytes are available
+     *
+     * @throws IOException exception on error
+     */
+    private void nextBuffer() throws IOException
+    {
+        long startNanos = System.nanoTime();
+        try
+        {
+            // Block on queue until next buffer available
+            Duration timeout = source.timeout();
+            if (timeout != null && timeout.getSeconds() > 0)
+            {
+                currentBuffer = queue.poll(timeout.getSeconds(), TimeUnit.SECONDS);
+                if (currentBuffer == null)
+                {
+                    throw new IOException(timeoutException(timeout));
+                }
+            }
+            else
+            {
+                currentBuffer = queue.take();
+            }
+        }
+        catch (InterruptedException exception)
+        {
+            Thread.currentThread().interrupt();
+            throw new RuntimeException(exception);
+        }
+        long nanosBlocked = System.nanoTime() - startNanos;
+        timeBlockedNanos += nanosBlocked;  // Measure time spent blocking for monitoring
+        stats.inputStreamTimeBlocked(source, nanosBlocked);
+
+        length = currentBuffer.readableBytes();
+        state = StreamState.Reading;
+        position = 0;
+        if (currentBuffer == null)
+        {
+            throw new IOException("Obtained a null buffer from the queue");
+        }
+    }
+
+    /**
+     * When reading from the InputStream first check the state, if stream is already closed or we need to
+     * pop off the next buffer from the {@link LinkedBlockingQueue}
+     *
+     * @return -1 if we have reached the end of the InputStream or 0 if still open
+     * @throws IOException throw IOException if stream is already closed
+     */
+    private int checkState() throws IOException
+    {
+        switch (state)
+        {
+            case Closed:
+                throw new IOException("Stream is closed");
+            case End:
+                return -1;
+            case Init:
+                // First request: start requesting bytes & schedule timeout
+                requestMore();
+                state = StreamState.NextBuffer;
+            case NextBuffer:
+                nextBuffer();
+                if (currentBuffer == END_MARKER)
+                {
+                    return handleEndMarker();
+                }
+                else if (currentBuffer == FINISHED_MARKER)
+                {
+                    return handleFinishedMarker();
+                }
+                else if (currentBuffer == ERROR_MARKER)
+                {
+                    throw new IOException(throwable);
+                }
+            default:
+                // Do nothing
+        }
+        return 0;
+    }
+
+    /**
+     * Handle finished marker returned in the queue, indicating all bytes from source have been requested
+     * and input stream can close
+     *
+     * @return always return -1 as stream is closed
+     */
+    private int handleFinishedMarker()
+    {
+        releaseBuffer();
+        end();
+        stats.inputStreamEndBuffer(source);
+        return -1;
+    }
+
+    /**
+     * Handle end marker returned in the queue, indicating previous request has finished
+     *
+     * @return -1 if we have reached the end of the InputStream or 0 if still open
+     * @throws IOException throw IOException if stream is already closed
+     */
+    private int handleEndMarker() throws IOException
+    {
+        if (skipping)
+        {
+            return -1;
+        }
+        releaseBuffer();
+        maybeRequestMore();
+        state = StreamState.NextBuffer;
+        return checkState();
+    }
+
+    /**
+     * Reached the end of the InputStream and all bytes have been read
+     */
+    private void end()
+    {
+        state = StreamState.End;
+        stats.inputStreamEnd(source, System.nanoTime() - startTimeNanos, timeBlockedNanos);
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/streaming/SSTableSource.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/streaming/SSTableSource.java
new file mode 100644
index 0000000..42af976
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/streaming/SSTableSource.java
@@ -0,0 +1,102 @@
+/*
+ * 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.cassandra.spark.utils.streaming;
+
+import java.time.Duration;
+
+import org.apache.cassandra.spark.data.FileType;
+import org.apache.cassandra.spark.data.SSTable;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * SSTableSource to asynchronously provide bytes to StreamConsumer when requested
+ *
+ * @param <T> SSTable type
+ */
+public interface SSTableSource<T extends SSTable>
+{
+    long DEFAULT_MAX_BUFFER_SIZE = 6291460L;
+    long DEFAULT_CHUNK_BUFFER_SIZE = 4194300L;
+
+    /**
+     * Asynchronously request bytes for the SSTable file component in the range start-end, and pass on to the StreamConsumer when available.
+     * The start-end range is inclusive.
+     *
+     * @param start    the start of the bytes range
+     * @param end      the end of the bytes range
+     * @param consumer the StreamConsumer to return the bytes to when the request is complete
+     */
+    void request(long start, long end, StreamConsumer consumer);
+
+    /**
+     * @return SSTable this source refers to
+     */
+    T sstable();
+
+    /**
+     * @return the SSTable file component type this source refers to
+     */
+    FileType fileType();
+
+    /**
+     * The total size in bytes of the SSTable file component
+     *
+     * @return the file size, in bytes
+     */
+    long size();
+
+    /**
+     * @return the max bytes the {@code org.apache.cassandra.spark.utils.streaming.SSTableInputStream} can buffer at one time
+     */
+    default long maxBufferSize()
+    {
+        return DEFAULT_MAX_BUFFER_SIZE;
+    }
+
+    /**
+     * @return the chunk size in bytes requested when {@link SSTableSource#request(long, long, StreamConsumer)} is called
+     */
+    default long chunkBufferSize()
+    {
+        return DEFAULT_CHUNK_BUFFER_SIZE;
+    }
+
+    /**
+     * For CommitLogs we may only need to read the header to determine if we can skip or not
+     * Override this value to reduce bytes requested in first request to read the header.
+     * NOTE: this is a best effort to reduce wastefully reading more bytes than the header, but
+     * the header might be variable length (e.g. for encrypted CommitLogs) in which case the header may be larger.
+     *
+     * @return the initial header size in bytes
+     */
+    default long headerChunkSize()
+    {
+        return chunkBufferSize();
+    }
+
+    /**
+     * @return the number of seconds with no activity before timing out the InputStream, null to disable timeouts
+     */
+    @Nullable
+    default Duration timeout()
+    {
+        return null;  // Disabled by default
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/streaming/StreamBuffer.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/streaming/StreamBuffer.java
new file mode 100644
index 0000000..badc732
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/streaming/StreamBuffer.java
@@ -0,0 +1,83 @@
+/*
+ * 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.cassandra.spark.utils.streaming;
+
+import java.nio.ByteBuffer;
+
+/**
+ * A generic wrapper around bytes to allow for on/off-heap byte arrays,
+ * whichever the underlying {@link SSTableSource} implementation uses
+ */
+public interface StreamBuffer
+{
+    void getBytes(int index, ByteBuffer destination, int length);
+
+    void getBytes(int index, byte[] destination, int destinationIndex, int length);
+
+    byte getByte(int index);
+
+    int readableBytes();
+
+    void release();
+
+    static ByteArrayWrapper wrap(byte[] bytes)
+    {
+        return new ByteArrayWrapper(bytes);
+    }
+
+    class ByteArrayWrapper implements StreamBuffer
+    {
+        private final byte[] bytes;
+
+        private ByteArrayWrapper(byte[] bytes)
+        {
+            this.bytes = bytes;
+        }
+
+        @Override
+        public void getBytes(int index, ByteBuffer destination, int length)
+        {
+            destination.put(bytes, index, length);
+        }
+
+        @Override
+        public void getBytes(int index, byte[] destination, int destinationIndex, int length)
+        {
+            System.arraycopy(bytes, index, destination, destinationIndex, length);
+        }
+
+        @Override
+        public byte getByte(int index)
+        {
+            return bytes[index];
+        }
+
+        @Override
+        public int readableBytes()
+        {
+            return bytes.length;
+        }
+
+        @Override
+        public void release()
+        {
+        }
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/streaming/StreamConsumer.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/streaming/StreamConsumer.java
new file mode 100644
index 0000000..1cd512a
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/streaming/StreamConsumer.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.cassandra.spark.utils.streaming;
+
+public interface StreamConsumer
+{
+    /**
+     * Called when {@link SSTableSource} completes a request and passes on the underlying bytes
+     *
+     * NOTE: This can be called multiple times after a single {@link SSTableSource#request(long, long, StreamConsumer)}
+     *
+     * @param buffer StreamBuffer wrapping the bytes
+     */
+    void onRead(StreamBuffer buffer);
+
+    /**
+     * Called when {@link SSTableSource} has finished calling onRead for the last time
+     * after {@link SSTableSource#request(long, long, StreamConsumer)} was called
+     *
+     * NOTE: {@link StreamConsumer#onRead(StreamBuffer)} may be called zero or more times
+     *       before {@link StreamConsumer#onEnd()} is called
+     */
+    void onEnd();
+
+    /**
+     * Called when {@link SSTableSource} fails for any reason to request the byte range
+     *
+     * @param throwable throwable
+     */
+    void onError(Throwable throwable);
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/test/TestSSTable.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/test/TestSSTable.java
new file mode 100644
index 0000000..423ea5a
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/test/TestSSTable.java
@@ -0,0 +1,133 @@
+/*
+ * 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.cassandra.spark.utils.test;
+
+import java.io.BufferedInputStream;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import org.apache.cassandra.spark.data.FileType;
+import org.apache.cassandra.spark.data.SSTable;
+import org.jetbrains.annotations.Nullable;
+
+public final class TestSSTable extends SSTable
+{
+    private final Path dataFile;
+
+    private TestSSTable(Path dataFile)
+    {
+        this.dataFile = dataFile;
+    }
+
+    public static SSTable at(Path dataFile)
+    {
+        return new TestSSTable(dataFile);
+    }
+
+    public static List<SSTable> at(Path... dataFiles)
+    {
+        return Arrays.stream(dataFiles)
+                     .map(TestSSTable::at)
+                     .collect(Collectors.toList());
+    }
+
+    private static Stream<Path> list(Path directory, FileType type) throws IOException
+    {
+        return Files.list(directory)
+                    .filter(path -> path.getFileName().toString().endsWith("-" + type.getFileSuffix()));
+    }
+
+    public static long countIn(Path directory) throws IOException
+    {
+        return list(directory, FileType.DATA)
+                .count();
+    }
+
+    public static List<SSTable> allIn(Path directory) throws IOException
+    {
+        return list(directory, FileType.DATA)
+                .map(TestSSTable::at)
+                .collect(Collectors.toList());
+    }
+
+    public static SSTable firstIn(Path directory) throws IOException
+    {
+        return list(directory, FileType.DATA)
+                .findFirst()
+                .map(TestSSTable::at)
+                .orElseThrow(FileNotFoundException::new);
+    }
+
+    @VisibleForTesting
+    public static Path firstIn(Path directory, FileType type) throws IOException
+    {
+        return list(directory, type)
+                .findFirst()
+                .orElseThrow(FileNotFoundException::new);
+    }
+
+    @Nullable
+    @Override
+    protected InputStream openInputStream(FileType fileType)
+    {
+        Path filePath = FileType.resolveComponentFile(fileType, dataFile);
+        try
+        {
+            return filePath != null ? new BufferedInputStream(new FileInputStream(filePath.toFile())) : null;
+        }
+        catch (FileNotFoundException exception)
+        {
+            return null;
+        }
+    }
+
+    public boolean isMissing(FileType fileType)
+    {
+        return FileType.resolveComponentFile(fileType, dataFile) == null;
+    }
+
+    @Override
+    public String getDataFileName()
+    {
+        return dataFile.getFileName().toString();
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return dataFile.hashCode();
+    }
+
+    @Override
+    public boolean equals(Object other)
+    {
+        return other instanceof TestSSTable && this.dataFile.equals(((TestSSTable) other).dataFile);
+    }
+}
diff --git a/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/test/TestSchema.java b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/test/TestSchema.java
new file mode 100644
index 0000000..f5022f4
--- /dev/null
+++ b/cassandra-bridge/src/main/java/org/apache/cassandra/spark/utils/test/TestSchema.java
@@ -0,0 +1,879 @@
+/*
+ * 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.cassandra.spark.utils.test;
+
+import java.math.BigDecimal;
+import java.math.RoundingMode;
+import java.nio.file.Path;
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Consumer;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import java.util.stream.Stream;
+
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.bridge.BigNumberConfig;
+import org.apache.cassandra.bridge.CassandraBridge;
+import org.apache.cassandra.bridge.CassandraVersion;
+import org.apache.cassandra.bridge.RangeTombstone;
+import org.apache.cassandra.spark.config.SchemaFeature;
+import org.apache.cassandra.spark.config.SchemaFeatureSet;
+import org.apache.cassandra.spark.data.CqlField;
+import org.apache.cassandra.spark.data.CqlTable;
+import org.apache.cassandra.spark.data.ReplicationFactor;
+import org.apache.cassandra.spark.data.partitioner.Partitioner;
+import org.apache.cassandra.spark.utils.ComparisonUtils;
+import org.apache.cassandra.spark.utils.RandomUtils;
+import org.apache.cassandra.spark.utils.TemporaryDirectory;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+import org.apache.spark.sql.types.StructType;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Helper class to create and test various schemas
+ */
+public final class TestSchema
+{
+    @SuppressWarnings("SameParameterValue")
+    public static class Builder
+    {
+        private String keyspace = null;
+        private String table = null;
+        private final List<CqlField> partitionKeys = new ArrayList<>();
+        private final List<CqlField> clusteringKeys = new ArrayList<>();
+        private final List<CqlField> columns = new ArrayList<>();
+        private final List<CqlField.SortOrder> sortOrders = new ArrayList<>();
+        private List<String> insertFields = null;
+        private List<String> deleteFields;
+        private int minCollectionSize = 16;
+        private Integer blobSize = null;
+        private boolean withCompression = true;
+
+        public Builder withKeyspace(String keyspace)
+        {
+            this.keyspace = keyspace;
+            return this;
+        }
+
+        public Builder withTable(String table)
+        {
+            this.table = table;
+            return this;
+        }
+
+        public Builder withPartitionKey(String name, CqlField.CqlType type)
+        {
+            partitionKeys.add(new CqlField(true, false, false, name, type, 0));
+            return this;
+        }
+
+        public Builder withClusteringKey(String name, CqlField.CqlType type)
+        {
+            clusteringKeys.add(new CqlField(false, true, false, name, type, 0));
+            return this;
+        }
+
+        public Builder withStaticColumn(String name, CqlField.CqlType type)
+        {
+            columns.add(new CqlField(false, false, true, name, type, 0));
+            return this;
+        }
+
+        public Builder withColumn(String name, CqlField.CqlType type)
+        {
+            columns.add(new CqlField(false, false, false, name, type, 0));
+            return this;
+        }
+
+        public Builder withSortOrder(CqlField.SortOrder sortOrder)
+        {
+            sortOrders.add(sortOrder);
+            return this;
+        }
+
+        public Builder withInsertFields(String... fields)
+        {
+            insertFields = Arrays.asList(fields);
+            return this;
+        }
+
+        public Builder withDeleteFields(String... fields)
+        {
+            deleteFields = Arrays.asList(fields);
+            return this;
+        }
+
+        public Builder withMinCollectionSize(int minCollectionSize)
+        {
+            this.minCollectionSize = minCollectionSize;
+            return this;
+        }
+
+        public Builder withCompression(boolean withCompression)
+        {
+            this.withCompression = withCompression;
+            return this;
+        }
+
+        // Override blob size
+        public Builder withBlobSize(int blobSize)
+        {
+            this.blobSize = blobSize;
+            return this;
+        }
+
+        public TestSchema build()
+        {
+            if (!partitionKeys.isEmpty())
+            {
+                return new TestSchema(
+                        keyspace != null ? keyspace : "keyspace_" + UUID.randomUUID().toString().replaceAll("-", ""),
+                        table != null ? table : "table_" + UUID.randomUUID().toString().replaceAll("-", ""),
+                        IntStream.range(0, partitionKeys.size())
+                                 .mapToObj(index -> partitionKeys.get(index).cloneWithPosition(index))
+                                 .sorted()
+                                 .collect(Collectors.toList()),
+                        IntStream.range(0, clusteringKeys.size())
+                                 .mapToObj(index -> clusteringKeys.get(index).cloneWithPosition(partitionKeys.size() + index))
+                                 .sorted()
+                                 .collect(Collectors.toList()),
+                        IntStream.range(0, columns.size())
+                                 .mapToObj(index -> columns.get(index).cloneWithPosition(partitionKeys.size() + clusteringKeys.size() + index))
+                                 .sorted(Comparator.comparing(CqlField::name))
+                                 .collect(Collectors.toList()),
+                        sortOrders,
+                        insertFields,
+                        deleteFields,
+                        minCollectionSize,
+                        blobSize,
+                        withCompression);
+            }
+            else
+            {
+                throw new IllegalArgumentException("Need at least one partition key");
+            }
+        }
+    }
+
+    @NotNull
+    public final String keyspace;
+    public final String table;
+    public final String createStatement;
+    public final String insertStatement;
+    public final String updateStatement;
+    public final String deleteStatement;
+    public final List<CqlField> partitionKeys;
+    public final List<CqlField> clusteringKeys;
+    final List<CqlField> allFields;
+    public final Set<CqlField.CqlUdt> udts;
+    private final Map<String, Integer> fieldPositions;
+    @Nullable
+    private CassandraVersion version = null;
+    private final int minCollectionSize;
+    private final Integer blobSize;
+
+    public static Builder builder()
+    {
+        return new Builder();
+    }
+
+    public static Builder basicBuilder(CassandraBridge bridge)
+    {
+        return TestSchema.builder()
+                         .withPartitionKey("a", bridge.aInt())
+                         .withClusteringKey("b", bridge.aInt())
+                         .withColumn("c", bridge.aInt());
+    }
+
+    public static TestSchema basic(CassandraBridge bridge)
+    {
+        return basicBuilder(bridge).build();
+    }
+
+    // CHECKSTYLE IGNORE: Constructor with many parameters
+    private TestSchema(@NotNull String keyspace,
+                       @NotNull String table,
+                       List<CqlField> partitionKeys,
+                       List<CqlField> clusteringKeys,
+                       List<CqlField> columns,
+                       List<CqlField.SortOrder> sortOrders,
+                       @Nullable List<String> insertOverrides,
+                       @Nullable List<String> deleteFields,
+                       int minCollectionSize,
+                       @Nullable Integer blobSize,
+                       boolean withCompression)
+    {
+        this.keyspace = keyspace;
+        this.table = table;
+        this.partitionKeys = partitionKeys;
+        this.clusteringKeys = clusteringKeys;
+        this.minCollectionSize = minCollectionSize;
+        this.blobSize = blobSize;
+        this.allFields = buildAllFields(partitionKeys, clusteringKeys, columns);
+        this.fieldPositions = calculateFieldPositions(allFields);
+        this.createStatement = buildCreateStatement(columns, sortOrders, withCompression);
+        this.insertStatement = buildInsertStatement(columns, insertOverrides);
+        this.updateStatement = buildUpdateStatement();
+        this.deleteStatement = buildDeleteStatement(deleteFields);
+        this.udts = getUdtsFromFields();
+    }
+
+    // We take allFields as a parameter here to ensure it's been created before use
+    @NotNull
+    private Map<String, Integer> calculateFieldPositions(@NotNull List<CqlField> allFields)
+    {
+        return allFields.stream().collect(Collectors.toMap(CqlField::name, CqlField::position));
+    }
+
+    @NotNull
+    private List<CqlField> buildAllFields(List<CqlField> partitionKeys,
+                                          List<CqlField> clusteringKeys,
+                                          List<CqlField> columns)
+    {
+        List<CqlField> allFields = new ArrayList<>(partitionKeys.size() + clusteringKeys.size() + columns.size());
+        allFields.addAll(partitionKeys);
+        allFields.addAll(clusteringKeys);
+        allFields.addAll(columns);
+        Collections.sort(allFields);
+        return allFields;
+    }
+
+    @NotNull
+    private Set<CqlField.CqlUdt> getUdtsFromFields()
+    {
+        return allFields.stream()
+                        .map(field -> field.type().udts())
+                        .flatMap(Collection::stream)
+                        .collect(Collectors.toSet());
+    }
+
+    private String buildDeleteStatement(@Nullable List<String> deleteFields)
+    {
+        StringBuilder deleteStmtBuilder = new StringBuilder().append("DELETE FROM ")
+                                                             .append(keyspace)
+                                                             .append(".")
+                                                             .append(table)
+                                                             .append(" WHERE ");
+        if (deleteFields != null)
+        {
+            deleteStmtBuilder.append(deleteFields.stream()
+                                                 .map(override -> override + " ?")
+                                                 .collect(Collectors.joining(" AND ")));
+        }
+        else
+        {
+            deleteStmtBuilder.append(allFields.stream()
+                                              .map(field -> field.name() + " = ?")
+                                              .collect(Collectors.joining(" AND ")));
+        }
+        return deleteStmtBuilder.append(";")
+                                .toString();
+    }
+
+    private String buildUpdateStatement()
+    {
+        StringBuilder updateStmtBuilder = new StringBuilder("UPDATE ").append(keyspace)
+                                                                      .append(".")
+                                                                      .append(table)
+                                                                      .append(" SET ");
+        updateStmtBuilder.append(allFields.stream()
+                                          .sorted()
+                                          .filter(field -> !field.isPartitionKey() && !field.isClusteringColumn())
+                                          .map(field -> field.name() + " = ?")
+                                          .collect(Collectors.joining(", ")));
+        updateStmtBuilder.append(" WHERE ");
+        updateStmtBuilder.append(allFields.stream()
+                                          .sorted()
+                                          .filter(field -> field.isPartitionKey() || field.isClusteringColumn())
+                                          .map(field -> field.name() + " = ?")
+                                          .collect(Collectors.joining(" and ")));
+        return updateStmtBuilder.append(";")
+                                .toString();
+    }
+
+    private String buildInsertStatement(List<CqlField> columns, @Nullable List<String> insertOverrides)
+    {
+        StringBuilder insertStmtBuilder = new StringBuilder().append("INSERT INTO ")
+                                                             .append(keyspace)
+                                                             .append(".")
+                                                             .append(table)
+                                                             .append(" (");
+        if (insertOverrides != null)
+        {
+            insertStmtBuilder.append(String.join(", ", insertOverrides))
+                             .append(") VALUES (")
+                             .append(insertOverrides.stream()
+                                                    .map(override -> "?")
+                                                    .collect(Collectors.joining(", ")));
+        }
+        else
+        {
+            insertStmtBuilder.append(allFields.stream()
+                                              .sorted()
+                                              .map(CqlField::name)
+                                              .collect(Collectors.joining(", ")))
+                             .append(") VALUES (")
+                             .append(Stream.of(partitionKeys, clusteringKeys, columns)
+                                           .flatMap(Collection::stream)
+                                           .sorted()
+                                           .map(field -> "?")
+                                           .collect(Collectors.joining(", ")));
+        }
+        return insertStmtBuilder.append(");")
+                                .toString();
+    }
+
+    private String buildCreateStatement(List<CqlField> columns,
+                                        List<CqlField.SortOrder> sortOrders,
+                                        boolean withCompression)
+    {
+        StringBuilder createStmtBuilder = new StringBuilder().append("CREATE TABLE ")
+                                                             .append(keyspace)
+                                                             .append(".")
+                                                             .append(table)
+                                                             .append(" (");
+        for (CqlField field : Stream.of(partitionKeys, clusteringKeys, columns)
+                                    .flatMap(Collection::stream)
+                                    .sorted()
+                                    .collect(Collectors.toList()))
+        {
+            createStmtBuilder.append(field.name())
+                             .append(" ")
+                             .append(field.cqlTypeName())
+                             .append(field.isStaticColumn() ? " static" : "")
+                             .append(", ");
+        }
+
+        createStmtBuilder.append("PRIMARY KEY((")
+                         .append(partitionKeys.stream()
+                                              .map(CqlField::name)
+                                              .collect(Collectors.joining(", ")))
+                         .append(")");
+
+        if (!clusteringKeys.isEmpty())
+        {
+            createStmtBuilder.append(", ")
+                             .append(clusteringKeys.stream()
+                                                   .map(CqlField::name)
+                                                   .collect(Collectors.joining(", ")));
+        }
+
+        createStmtBuilder.append("))");
+
+        if (!sortOrders.isEmpty())
+        {
+            createStmtBuilder.append(" WITH CLUSTERING ORDER BY (");
+            for (int sortOrder = 0; sortOrder < sortOrders.size(); sortOrder++)
+            {
+                createStmtBuilder.append(clusteringKeys.get(sortOrder).name())
+                                 .append(" ")
+                                 .append(sortOrders.get(sortOrder).toString());
+                if (sortOrder < sortOrders.size() - 1)
+                {
+                    createStmtBuilder.append(", ");
+                }
+            }
+            createStmtBuilder.append(")");
+        }
+
+        if (!withCompression)
+        {
+            createStmtBuilder.append(" WITH compression = {'enabled':'false'}");
+        }
+
+        return createStmtBuilder.append(";")
+                                .toString();
+    }
+
+    public void setCassandraVersion(@NotNull CassandraVersion version)
+    {
+        this.version = version;
+    }
+
+    public CqlTable buildTable()
+    {
+        return new CqlTable(keyspace,
+                            table,
+                            createStatement,
+                            new ReplicationFactor(ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy,
+                                                  ImmutableMap.of("DC1", 3)),
+                            allFields,
+                            udts,
+                            0);
+    }
+
+    public void writeSSTable(TemporaryDirectory directory,
+                             CassandraBridge bridge,
+                             Partitioner partitioner,
+                             Consumer<CassandraBridge.Writer> writer)
+    {
+        writeSSTable(directory.path(), bridge, partitioner, writer);
+    }
+
+    public void writeSSTable(Path directory,
+                             CassandraBridge bridge,
+                             Partitioner partitioner,
+                             Consumer<CassandraBridge.Writer> writer)
+    {
+        writeSSTable(directory, bridge, partitioner, false, writer);
+    }
+
+    public void writeSSTable(Path directory,
+                             CassandraBridge bridge,
+                             Partitioner partitioner,
+                             boolean upsert,
+                             Consumer<CassandraBridge.Writer> writer)
+    {
+        bridge.writeSSTable(partitioner,
+                            keyspace,
+                            table,
+                            directory,
+                            createStatement,
+                            insertStatement,
+                            updateStatement,
+                            upsert,
+                            udts,
+                            writer);
+    }
+
+    public void writeTombstoneSSTable(Path directory,
+                                      CassandraBridge bridge,
+                                      Partitioner partitioner,
+                                      Consumer<CassandraBridge.Writer> writer)
+    {
+        bridge.writeTombstoneSSTable(partitioner, directory, createStatement, deleteStatement, writer);
+    }
+
+    public static StructType toStructType(CqlTable table, boolean addLastModificationTimeColumn)
+    {
+        StructType structType = new StructType();
+        for (CqlField field : table.fields())
+        {
+            structType = structType.add(field.name(), field.type().sparkSqlType(BigNumberConfig.DEFAULT));
+        }
+        if (addLastModificationTimeColumn)
+        {
+            structType = structType.add(SchemaFeatureSet.LAST_MODIFIED_TIMESTAMP.field());
+        }
+        // CDC jobs always add the updated_fields_indicator and is_update column
+        for (SchemaFeature feature : SchemaFeatureSet.ALL_CDC_FEATURES)
+        {
+            structType = structType.add(feature.field());
+        }
+        return structType;
+    }
+
+    public TestRow[] randomRows(int numRows)
+    {
+        return randomRows(numRows, 0);
+    }
+
+    @SuppressWarnings("SameParameterValue")
+    private TestRow[] randomRows(int numRows, int numTombstones)
+    {
+        TestSchema.TestRow[] testRows = new TestSchema.TestRow[numRows];
+        for (int testRow = 0; testRow < testRows.length; testRow++)
+        {
+            testRows[testRow] = randomRow(testRow < numTombstones);
+        }
+        return testRows;
+    }
+
+    public TestRow randomRow()
+    {
+        return randomRow(false);
+    }
+
+    private TestRow randomRow(boolean tombstone)
+    {
+        return randomRow(field -> tombstone && field.isValueColumn());
+    }
+
+    private TestRow randomRow(Predicate<CqlField> nullifiedFields)
+    {
+        Object[] values = new Object[allFields.size()];
+        for (CqlField field : allFields)
+        {
+            if (nullifiedFields.test(field))
+            {
+                values[field.position()] = null;
+            }
+            else
+            {
+                if (field.type().getClass().getSimpleName().equals("Blob") && blobSize != null)
+                {
+                    values[field.position()] = RandomUtils.randomByteBuffer(blobSize);
+                }
+                else
+                {
+                    values[field.position()] = field.type().randomValue(minCollectionSize);
+                }
+            }
+        }
+        return new TestRow(values);
+    }
+
+    public TestRow randomPartitionDelete()
+    {
+        return randomRow(field -> !field.isPartitionKey());
+    }
+
+    public TestRow toTestRow(InternalRow row)
+    {
+        if (row instanceof GenericInternalRow)
+        {
+            Object[] values = new Object[allFields.size()];
+            for (CqlField field : allFields)
+            {
+                values[field.position()] = field.type().sparkSqlRowValue((GenericInternalRow) row, field.position());
+            }
+            return new TestRow(values);
+        }
+        else
+        {
+            throw new IllegalStateException("Can only convert GenericInternalRow");
+        }
+    }
+
+    public TestRow toTestRow(Row row, Set<String> requiredColumns)
+    {
+        Object[] values = new Object[requiredColumns != null ? requiredColumns.size() : allFields.size()];
+        int skipped = 0;
+        for (CqlField field : allFields)
+        {
+            if (requiredColumns != null && !requiredColumns.contains(field.name()))
+            {
+                skipped++;
+                continue;
+            }
+            int position = field.position() - skipped;
+            values[position] = row.get(position) != null ? field.type().sparkSqlRowValue(row, position) : null;
+        }
+        return new TestRow(values);
+    }
+
+    @SuppressWarnings("SameParameterValue")
+    public final class TestRow implements CassandraBridge.IRow
+    {
+        private final Object[] values;
+        private boolean isTombstoned;
+        private boolean isInsert;
+        private List<RangeTombstone> rangeTombstones;
+
+        private TestRow(Object[] values)
+        {
+            this(values, false, true);
+        }
+
+        private TestRow(Object[] values, boolean isTombstoned, boolean isInsert)
+        {
+            this.values = values;
+            this.isTombstoned = isTombstoned;
+            this.isInsert = isInsert;
+        }
+
+        public void setRangeTombstones(List<RangeTombstone> rangeTombstones)
+        {
+            this.rangeTombstones = rangeTombstones;
+        }
+
+        @Override
+        public List<RangeTombstone> rangeTombstones()
+        {
+            return rangeTombstones;
+        }
+
+        public void delete()
+        {
+            isTombstoned = true;
+        }
+
+        public void fromUpdate()
+        {
+            isInsert = false;
+        }
+
+        public void fromInsert()
+        {
+            isInsert = true;
+        }
+
+        @Override
+        public boolean isDeleted()
+        {
+            return isTombstoned;
+        }
+
+        @Override
+        public boolean isInsert()
+        {
+            return isInsert;
+        }
+
+        public TestRow copy(String field, Object value)
+        {
+            return copy(getFieldPosition(field), value);
+        }
+
+        public TestRow copy(int position, Object value)
+        {
+            Object[] newValues = new Object[values.length];
+            System.arraycopy(values, 0, newValues, 0, values.length);
+            newValues[position] = value;
+            return new TestRow(newValues);
+        }
+
+        /**
+         * If a prune column filter is applied, convert expected TestRow to only include required columns
+         * so we can compare with row returned by Spark
+         *
+         * @param columns required columns, or null if no column selection criteria
+         * @return a TestRow containing only the required columns
+         */
+        public TestRow withColumns(@Nullable Set<String> columns)
+        {
+            if (columns == null)
+            {
+                return this;
+            }
+            Object[] result = new Object[columns.size()];
+            int skipped = 0;
+            for (CqlField field : allFields)
+            {
+                if (!columns.contains(field.name()))
+                {
+                    skipped++;
+                    continue;
+                }
+                result[field.position() - skipped] = values[field.position()];
+            }
+            return new TestRow(result);
+        }
+
+        public TestRow nullifyUnsetColumn()
+        {
+            Object[] newValues = new Object[values.length];
+            System.arraycopy(values, 0, newValues, 0, values.length);
+            for (int value = 0; value < newValues.length; value++)
+            {
+                if (newValues[value] == CassandraBridge.UNSET_MARKER)
+                {
+                    newValues[value] = null;
+                }
+            }
+            return new TestRow(newValues);
+        }
+
+        public Object[] rawValues(int start, int end)
+        {
+            assert start <= end && end <= values.length
+                : String.format("start: %s, end: %s", version, start, end);
+            Object[] result = new Object[end - start];
+            System.arraycopy(values, start, result, 0, end - start);
+            return result;
+        }
+
+        public Object[] allValues()
+        {
+            return values(0, values.length);
+        }
+
+        // Start inclusive, end exclusive
+        public Object[] values(int start, int end)
+        {
+            // NOTE: CassandraBridge must be set before calling this class,
+            //       so we can convert 4.0 Date type to LocalDate to be used in CQLSSTableWriter
+            assert version != null && start <= end && end <= values.length
+                : String.format("version: %s, start: %s, end: %s", version, start, end);
+            Object[] result = new Object[end - start];
+            for (int sourceIndex = start, destinationIndex = 0; sourceIndex < end; sourceIndex++, destinationIndex++)
+            {
+                result[destinationIndex] = convertForCqlWriter(getType(sourceIndex), values[sourceIndex]);
+            }
+            return result;
+        }
+
+        private Object convertForCqlWriter(CqlField.CqlType type, Object value)
+        {
+            return type.convertForCqlWriter(value, version);
+        }
+
+        public CqlField.CqlType getType(int position)
+        {
+            if (0 <= position && position < allFields.size())
+            {
+                return allFields.get(position).type();
+            }
+            else
+            {
+                throw new IllegalStateException("Unknown field at position: " + position);
+            }
+        }
+
+        public boolean isNull(String field)
+        {
+            return get(field) == null;
+        }
+
+        public String getString(String field)
+        {
+            return (String) get(field);
+        }
+
+        public UUID getUUID(String field)
+        {
+            return (UUID) get(field);
+        }
+
+        public Long getLong(String field)
+        {
+            return (Long) get(field);
+        }
+
+        public Integer getInteger(String field)
+        {
+            return (Integer) get(field);
+        }
+
+        public Object get(String field)
+        {
+            return get(getFieldPosition(field));
+        }
+
+        private int getFieldPosition(String field)
+        {
+            return Objects.requireNonNull(fieldPositions.get(field), "Unknown field: " + field);
+        }
+
+        @Override
+        public Object get(int position)
+        {
+            return values[position];
+        }
+
+        public boolean isTombstone()
+        {
+            return allFields.stream()
+                            .filter(CqlField::isValueColumn)
+                            .allMatch(field -> values[field.position()] == null);
+        }
+
+        public String getKey()
+        {
+            StringBuilder str = new StringBuilder();
+            for (int key = 0; key < partitionKeys.size() + clusteringKeys.size(); key++)
+            {
+                CqlField.CqlType type = key < partitionKeys.size()
+                        ? partitionKeys.get(key).type()
+                        : clusteringKeys.get(key - partitionKeys.size()).type();
+                str.append(toString(type, get(key))).append(":");
+            }
+            return str.toString();
+        }
+
+        private String toString(CqlField.CqlType type, Object key)
+        {
+            if (key instanceof BigDecimal)
+            {
+                return ((BigDecimal) key).setScale(8, RoundingMode.CEILING).toPlainString();
+            }
+            else if (key instanceof Timestamp)
+            {
+                return new Date(((Timestamp) key).getTime()).toString();
+            }
+            else if (key instanceof Object[])
+            {
+                return String.format("[%s]", Arrays.stream((Object[]) key)
+                                                   .map(value -> toString(type, value))
+                                                   .collect(Collectors.joining(", ")));
+            }
+            else if (key instanceof Map)
+            {
+                CqlField.CqlType innerType = getFrozenInnerType(type);
+                if (innerType instanceof CqlField.CqlMap)
+                {
+                    CqlField.CqlMap mapType = (CqlField.CqlMap) innerType;
+                    return ((Map<?, ?>) key).entrySet()
+                            .stream()
+                            .sorted((Comparator<Map.Entry<?, ?>>) (first, second) ->
+                                    mapType.keyType().compare(first.getKey(), second.getKey()))
+                            .map(Map.Entry::getValue)
+                            .collect(Collectors.toList())
+                            .toString();
+                }
+                return ((Map<?, ?>) key).entrySet().stream().collect(
+                        Collectors.toMap(entry -> toString(innerType, entry.getKey()),
+                                         entry -> toString(innerType, entry.getValue()))).toString();
+            }
+            else if (key instanceof Collection)
+            {
+                CqlField.CqlType innerType = ((CqlField.CqlCollection) getFrozenInnerType(type)).type();
+                return ((Collection<?>) key).stream()
+                                            .sorted(innerType)
+                                            .map(value -> toString(innerType, value))
+                                            .collect(Collectors.toList()).toString();
+            }
+            return key != null ? key.toString() : "null";
+        }
+
+        private CqlField.CqlType getFrozenInnerType(CqlField.CqlType type)
+        {
+            if (type instanceof CqlField.CqlFrozen)
+            {
+                return getFrozenInnerType(((CqlField.CqlFrozen) type).inner());
+            }
+            return type;
+        }
+
+        @Override
+        public String toString()
+        {
+            return String.format("[%s]", IntStream.range(0, values.length)
+                                                  .mapToObj(index -> toString(allFields.get(index).type(), values[index]))
+                                                  .collect(Collectors.joining(", ")));
+        }
+
+        public int hashCode()
+        {
+            return Objects.hash(values);
+        }
+
+        public boolean equals(Object other)
+        {
+            return other instanceof TestRow && ComparisonUtils.equals(this.values, ((TestRow) other).values);
+        }
+    }
+}
diff --git a/cassandra-bridge/src/main/scala-2.11-spark-2/org/apache/cassandra/spark/utils/ScalaConversionUtils.java b/cassandra-bridge/src/main/scala-2.11-spark-2/org/apache/cassandra/spark/utils/ScalaConversionUtils.java
new file mode 100644
index 0000000..5cfb85b
--- /dev/null
+++ b/cassandra-bridge/src/main/scala-2.11-spark-2/org/apache/cassandra/spark/utils/ScalaConversionUtils.java
@@ -0,0 +1,56 @@
+/*
+ * 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.cassandra.spark.utils;
+
+import java.util.List;
+
+import scala.collection.JavaConversions;
+import scala.collection.mutable.Seq;
+
+/**
+ * Compatibility layer for scala conversions
+ */
+public final class ScalaConversionUtils
+{
+    private ScalaConversionUtils()
+    {
+        throw new IllegalStateException(getClass() + " is static utility class and shall not be instantiated");
+    }
+
+    public static <A> java.lang.Iterable<A> asJavaIterable(scala.collection.Iterable<A> iterable)
+    {
+        return JavaConversions.asJavaIterable(iterable);
+    }
+
+    public static <A> scala.collection.Iterator<A> asScalaIterator(java.util.Iterator<A> iterator)
+    {
+        return JavaConversions.asScalaIterator(iterator);
+    }
+
+    public static <A, B> java.util.Map<A, B> mapAsJavaMap(scala.collection.Map<A, B> map)
+    {
+        return JavaConversions.mapAsJavaMap(map);
+    }
+
+    public static <A> List<A> mutableSeqAsJavaList(Seq<A> seq)
+    {
+        return JavaConversions.mutableSeqAsJavaList(seq);
+    }
+}
diff --git a/cassandra-bridge/src/main/scala-2.12-spark-2/org/apache/cassandra/spark/utils/ScalaConversionUtils.java b/cassandra-bridge/src/main/scala-2.12-spark-2/org/apache/cassandra/spark/utils/ScalaConversionUtils.java
new file mode 100644
index 0000000..23eabb3
--- /dev/null
+++ b/cassandra-bridge/src/main/scala-2.12-spark-2/org/apache/cassandra/spark/utils/ScalaConversionUtils.java
@@ -0,0 +1,56 @@
+/*
+ * 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.cassandra.spark.utils;
+
+import java.util.List;
+
+import scala.collection.JavaConverters;
+import scala.collection.mutable.Seq;
+
+/**
+ * Compatibility layer for scala conversions
+ */
+public final class ScalaConversionUtils
+{
+    private ScalaConversionUtils()
+    {
+        throw new IllegalStateException(getClass() + " is static utility class and shall not be instantiated");
+    }
+
+    public static <A> java.lang.Iterable<A> asJavaIterable(scala.collection.Iterable<A> iterable)
+    {
+        return JavaConverters.asJavaIterable(iterable);
+    }
+
+    public static <A> scala.collection.Iterator<A> asScalaIterator(java.util.Iterator<A> iterator)
+    {
+        return JavaConverters.asScalaIterator(iterator);
+    }
+
+    public static <A, B> java.util.Map<A, B> mapAsJavaMap(scala.collection.Map<A, B> map)
+    {
+        return JavaConverters.mapAsJavaMap(map);
+    }
+
+    public static <A> List<A> mutableSeqAsJavaList(Seq<A> seq)
+    {
+        return JavaConverters.mutableSeqAsJavaList(seq);
+    }
+}
diff --git a/cassandra-bridge/src/main/scala-2.12-spark-3/org/apache/cassandra/spark/utils/ScalaConversionUtils.java b/cassandra-bridge/src/main/scala-2.12-spark-3/org/apache/cassandra/spark/utils/ScalaConversionUtils.java
new file mode 100644
index 0000000..23eabb3
--- /dev/null
+++ b/cassandra-bridge/src/main/scala-2.12-spark-3/org/apache/cassandra/spark/utils/ScalaConversionUtils.java
@@ -0,0 +1,56 @@
+/*
+ * 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.cassandra.spark.utils;
+
+import java.util.List;
+
+import scala.collection.JavaConverters;
+import scala.collection.mutable.Seq;
+
+/**
+ * Compatibility layer for scala conversions
+ */
+public final class ScalaConversionUtils
+{
+    private ScalaConversionUtils()
+    {
+        throw new IllegalStateException(getClass() + " is static utility class and shall not be instantiated");
+    }
+
+    public static <A> java.lang.Iterable<A> asJavaIterable(scala.collection.Iterable<A> iterable)
+    {
+        return JavaConverters.asJavaIterable(iterable);
+    }
+
+    public static <A> scala.collection.Iterator<A> asScalaIterator(java.util.Iterator<A> iterator)
+    {
+        return JavaConverters.asScalaIterator(iterator);
+    }
+
+    public static <A, B> java.util.Map<A, B> mapAsJavaMap(scala.collection.Map<A, B> map)
+    {
+        return JavaConverters.mapAsJavaMap(map);
+    }
+
+    public static <A> List<A> mutableSeqAsJavaList(Seq<A> seq)
+    {
+        return JavaConverters.mutableSeqAsJavaList(seq);
+    }
+}
diff --git a/cassandra-bridge/src/main/scala-2.13-spark-3/org/apache/cassandra/spark/utils/ScalaConversionUtils.java b/cassandra-bridge/src/main/scala-2.13-spark-3/org/apache/cassandra/spark/utils/ScalaConversionUtils.java
new file mode 100644
index 0000000..079f511
--- /dev/null
+++ b/cassandra-bridge/src/main/scala-2.13-spark-3/org/apache/cassandra/spark/utils/ScalaConversionUtils.java
@@ -0,0 +1,56 @@
+/*
+ * 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.cassandra.spark.utils;
+
+import java.util.List;
+
+import scala.collection.mutable.Seq;
+import scala.jdk.javaapi.CollectionConverters;
+
+/**
+ * Compatibility layer for scala conversions
+ */
+public final class ScalaConversionUtils
+{
+    private ScalaConversionUtils()
+    {
+        throw new IllegalStateException(getClass() + " is static utility class and shall not be instantiated");
+    }
+
+    public static <A> java.lang.Iterable<A> asJavaIterable(scala.collection.Iterable<A> iterable)
+    {
+        return CollectionConverters.asJava(iterable);
+    }
+
+    public static <A> scala.collection.Iterator<A> asScalaIterator(java.util.Iterator<A> iterator)
+    {
+        return CollectionConverters.asScala(iterator);
+    }
+
+    public static <K, V> java.util.Map<K, V> mapAsJavaMap(scala.collection.Map<K, V> map)
+    {
+        return CollectionConverters.asJava(map);
+    }
+
+    public static <A> List<A> mutableSeqAsJavaList(Seq<A> seq)
+    {
+        return CollectionConverters.asJava(seq);
+    }
+}
diff --git a/cassandra-bridge/src/test/java/org/apache/cassandra/bridge/BigNumberConfigImplTest.java b/cassandra-bridge/src/test/java/org/apache/cassandra/bridge/BigNumberConfigImplTest.java
new file mode 100644
index 0000000..c1187b8
--- /dev/null
+++ b/cassandra-bridge/src/test/java/org/apache/cassandra/bridge/BigNumberConfigImplTest.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.cassandra.bridge;
+
+import java.io.IOException;
+import java.util.Map;
+
+import com.google.common.collect.ImmutableMap;
+import org.junit.Test;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Unit tests for {@link BigNumberConfigImpl}
+ */
+public class BigNumberConfigImplTest
+{
+    private static final ObjectMapper MAPPER = new ObjectMapper();
+
+    @Test
+    public void testJsonSerialization() throws IOException
+    {
+        Map<String, BigNumberConfigImpl> map = ImmutableMap.of("field1", BigNumberConfigImpl.of(10, 4, 38, 19),
+                                                               "field2", BigNumberConfigImpl.of(10, 4, 38, 19));
+        String json = MAPPER.writeValueAsString(map);
+        Map<String, BigNumberConfigImpl> deserialized = BigNumberConfigImpl.build(json);
+        assertEquals(map, deserialized);
+    }
+}
diff --git a/cassandra-bridge/src/test/java/org/apache/cassandra/spark/utils/ArrayUtilsTest.java b/cassandra-bridge/src/test/java/org/apache/cassandra/spark/utils/ArrayUtilsTest.java
new file mode 100644
index 0000000..3685961
--- /dev/null
+++ b/cassandra-bridge/src/test/java/org/apache/cassandra/spark/utils/ArrayUtilsTest.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.cassandra.spark.utils;
+
+import java.util.function.Consumer;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import static org.apache.cassandra.spark.utils.ArrayUtils.retain;
+
+public class ArrayUtilsTest
+{
+    @Test
+    public void testRetain()
+    {
+        Object[] source = new Object[]{1, 2, 3, 4, 5};
+        Assert.assertArrayEquals(new Object[]{1, 2, 3}, retain(source, 0, 3));
+    }
+
+    @Test
+    public void testRetainThrows()
+    {
+        // Not using JUnit rule ExpectedException in order to assert multiple throwables
+        expectedThrows(() -> retain(null, 0, 1),
+                       throwable -> Assert.assertSame(IllegalArgumentException.class, throwable.getClass()));
+
+        expectedThrows(() -> retain(new Object[]{1, 2, 3}, -1, 1),
+                       throwable -> Assert.assertSame(IllegalArgumentException.class, throwable.getClass()));
+
+        expectedThrows(() -> retain(new Object[]{1, 2, 3}, 0, -1),
+                       throwable -> Assert.assertSame(IllegalArgumentException.class, throwable.getClass()));
+
+        expectedThrows(() -> retain(new Object[]{1, 2, 3}, 0, 5),
+                       throwable -> Assert.assertSame(IllegalArgumentException.class, throwable.getClass()));
+    }
+
+    private void expectedThrows(Runnable test, Consumer<Throwable> throwableVerifier)
+    {
+        try
+        {
+            test.run();
+        }
+        catch (Throwable throwable)
+        {
+            throwableVerifier.accept(throwable);
+        }
+    }
+}
diff --git a/cassandra-bridge/src/test/java/org/apache/cassandra/spark/utils/LoggerHelperTests.java b/cassandra-bridge/src/test/java/org/apache/cassandra/spark/utils/LoggerHelperTests.java
new file mode 100644
index 0000000..c95900e
--- /dev/null
+++ b/cassandra-bridge/src/test/java/org/apache/cassandra/spark/utils/LoggerHelperTests.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.cassandra.spark.utils;
+
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+public class LoggerHelperTests
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(LoggerHelperTests.class);
+
+    @Test
+    public void testBuildArgs()
+    {
+        LoggerHelper loggerHelper = new LoggerHelper(LOGGER, "a", "1", "b", "2", "c", "3");
+        Object[] additionalArgs = new Object[]{"d", "4", "e", "5", "f", "6"};
+        assertEquals("hello this is the log message a={} b={} c={} d={} e={} f={}", loggerHelper.logMsg("hello this is the log message", additionalArgs));
+        assertArrayEquals(new Object[]{"1", "2", "3", "4", "5", "6"}, loggerHelper.buildArguments(null, additionalArgs));
+        Throwable throwable = new RuntimeException("Error");
+        assertArrayEquals(new Object[]{"1", "2", "3", "4", "5", "6", throwable}, loggerHelper.buildArguments(throwable, additionalArgs));
+    }
+}
diff --git a/cassandra-bridge/src/test/java/org/apache/cassandra/spark/utils/ThrowableUtilTests.java b/cassandra-bridge/src/test/java/org/apache/cassandra/spark/utils/ThrowableUtilTests.java
new file mode 100644
index 0000000..9cd46c7
--- /dev/null
+++ b/cassandra-bridge/src/test/java/org/apache/cassandra/spark/utils/ThrowableUtilTests.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.cassandra.spark.utils;
+
+import java.io.IOException;
+import java.util.Objects;
+
+import org.junit.Test;
+
+import org.apache.cassandra.spark.exceptions.TransportFailureException;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+public class ThrowableUtilTests
+{
+    @Test
+    public void testNoNesting()
+    {
+        Throwable throwable = new RuntimeException();
+        assertEquals(throwable, ThrowableUtils.rootCause(throwable));
+    }
+
+    @Test
+    public void testNested()
+    {
+        Throwable throwable2 = new RuntimeException();
+        Throwable throwable1 = new RuntimeException(throwable2);
+        assertEquals(throwable2, ThrowableUtils.rootCause(throwable1));
+        assertEquals(throwable2, ThrowableUtils.rootCause(throwable2));
+    }
+
+    @Test
+    public void testMultiNested()
+    {
+        Throwable throwable4 = new RuntimeException();
+        Throwable throwable3 = new RuntimeException(throwable4);
+        Throwable throwable2 = new RuntimeException(throwable3);
+        Throwable throwable1 = new RuntimeException(throwable2);
+        assertEquals(throwable4, ThrowableUtils.rootCause(throwable1));
+        assertEquals(throwable4, ThrowableUtils.rootCause(throwable2));
+        assertEquals(throwable4, ThrowableUtils.rootCause(throwable3));
+        assertEquals(throwable4, ThrowableUtils.rootCause(throwable4));
+    }
+
+    @Test
+    public void testOfType()
+    {
+        IOException io = new IOException();
+        Throwable throwable = new RuntimeException(io);
+        assertEquals(io, ThrowableUtils.rootCause(throwable, IOException.class));
+        assertEquals(io, ThrowableUtils.rootCause(io, IOException.class));
+    }
+
+    @Test
+    public void testOfType2()
+    {
+        TransportFailureException exception = TransportFailureException.nonretryable(404);
+        Throwable throwable = new RuntimeException(exception);
+        assertEquals(exception, ThrowableUtils.rootCause(throwable, TransportFailureException.class));
+        assertTrue(Objects.requireNonNull(ThrowableUtils.rootCause(throwable, TransportFailureException.class)).isNotFound());
+        assertEquals(exception, ThrowableUtils.rootCause(exception, TransportFailureException.class));
+    }
+
+    @Test
+    public void testOfTypeNested()
+    {
+        Throwable throwable4 = new RuntimeException();
+        IOException io = new IOException(throwable4);
+        Throwable throwable3 = new RuntimeException(io);
+        Throwable throwable2 = new RuntimeException(throwable3);
+        Throwable throwable1 = new RuntimeException(throwable2);
+        assertEquals(io, ThrowableUtils.rootCause(throwable1, IOException.class));
+        assertEquals(io, ThrowableUtils.rootCause(throwable2, IOException.class));
+        assertNull(ThrowableUtils.rootCause(throwable4, IOException.class));
+    }
+
+    @Test
+    public void testOfTypeNotFound()
+    {
+        Throwable throwable = new RuntimeException();
+        assertNull(ThrowableUtils.rootCause(throwable, IOException.class));
+    }
+
+    @Test
+    public void testOfTypeNotExist()
+    {
+        Throwable throwable4 = new RuntimeException();
+        Throwable throwable3 = new RuntimeException(throwable4);
+        Throwable throwable2 = new RuntimeException(throwable3);
+        Throwable throwable1 = new RuntimeException(throwable2);
+        assertNull(ThrowableUtils.rootCause(throwable1, IOException.class));
+    }
+}
diff --git a/cassandra-four-zero/build.gradle b/cassandra-four-zero/build.gradle
new file mode 100644
index 0000000..7f6c4e5
--- /dev/null
+++ b/cassandra-four-zero/build.gradle
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+project(':cassandra-four-zero') {
+    apply(plugin: 'java-library')
+    apply(plugin: 'com.github.johnrengelman.shadow')
+
+    dependencies {
+        compileOnly(project(':cassandra-bridge'))
+
+        compileOnly(group: "${sparkGroupId}", name: "spark-core_${scalaMajorVersion}", version: "${project.rootProject.sparkVersion}")
+        compileOnly(group: "${sparkGroupId}", name: "spark-sql_${scalaMajorVersion}", version: "${project.rootProject.sparkVersion}")
+
+        implementation(group: 'org.apache.cassandra', name: 'cassandra-all', version: '4.0.2') {
+            // Exclude JNA libraries from the cassandra-all dependency tree because Spark has its own version
+            // and trying to load two different versions causes issues with the native libraries
+            exclude(group: 'net.java.dev.jna')
+            exclude(group: 'org.xerial.snappy', module: 'snappy-java')
+
+            // zstd causes dependency/classpath issues running in Spark if e.g. a user is using zstd to write compressed output file
+            // Spark already pulls in zstd so we can exclude here
+            exclude(group: 'com.github.luben', module: 'zstd-jni')
+        }
+
+        testImplementation(project(':cassandra-bridge'))
+
+        testImplementation(group: 'junit', name: 'junit', version: "${project.rootProject.junitVersion}")
+        testImplementation(group: 'org.quicktheories', name: 'quicktheories', version: "${project.rootProject.quickTheoriesVersion}")
+        testImplementation("org.mockito:mockito-core:${project.rootProject.mockitoVersion}")
+        testImplementation(group: "${sparkGroupId}", name: "spark-core_${scalaMajorVersion}", version: "${project.rootProject.sparkVersion}")
+        testImplementation(group: "${sparkGroupId}", name: "spark-sql_${scalaMajorVersion}", version: "${project.rootProject.sparkVersion}")
+        testImplementation(group: 'com.github.luben', name: 'zstd-jni', version: '1.5.0-4')
+
+        testRuntimeOnly(group: 'net.java.dev.jna', name: 'jna', version: "${jnaVersion}")
+        testRuntimeOnly(group: 'net.java.dev.jna', name: 'jna-platform', version: "${jnaVersion}")
+    }
+
+    jar {
+        enabled = false
+    }
+
+    shadowJar {
+        archiveFileName = 'four-zero.jar'  // Must match label in CassandraVersion (without extension)
+        zip64 = true
+
+        exclude('META-INF/maven/org/apache/logging/**')
+        exclude('META-INF/maven/org.apache.logging.log4j/**')
+        exclude('META-INF/maven/org.slf4j/**')
+        exclude('com/fasterxml/jackson/**')
+        exclude('net/openhft/chronicle/wire/**')
+        exclude('org/jboss/byteman/**')
+        exclude('org/junit/**')
+        exclude('org/quicktheories/**')
+        exclude('org/slf4j/**')
+        exclude('org/apache/commons/lang3/**')
+
+        // Exclude Google Guava's implementation of the Range<BigInteger> and related classes from the JAr,
+        // this way all implementations of Cassandra Bridge will fall back to using the exact same version.
+        exclude('com/google/common/collect/Range*')
+    }
+
+    task javadocJar(type: Jar, dependsOn: javadoc) {
+      classifier = 'javadoc'
+      from javadoc.destinationDir
+    }
+
+    artifacts {
+        javadocJar
+        shadowJar
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/bridge/CassandraBridgeImplementation.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/bridge/CassandraBridgeImplementation.java
new file mode 100644
index 0000000..7bdd4c1
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/bridge/CassandraBridgeImplementation.java
@@ -0,0 +1,1051 @@
+/*
+ * 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.cassandra.bridge;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.AbstractMap;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.function.BiConsumer;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.Serializer;
+import com.esotericsoftware.kryo.io.Input;
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.ParameterizedClass;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.db.Clustering;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.LivenessInfo;
+import org.apache.cassandra.db.Mutation;
+import org.apache.cassandra.db.commitlog.BufferingCommitLogReader;
+import org.apache.cassandra.db.commitlog.CommitLogSegmentManagerCDC;
+import org.apache.cassandra.db.commitlog.PartitionUpdateWrapper;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.ByteBufferAccessor;
+import org.apache.cassandra.db.marshal.CompositeType;
+import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.db.rows.BTreeRow;
+import org.apache.cassandra.db.rows.CellPath;
+import org.apache.cassandra.db.rows.Row;
+import org.apache.cassandra.db.rows.Rows;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.dht.RandomPartitioner;
+import org.apache.cassandra.io.compress.ICompressor;
+import org.apache.cassandra.io.compress.LZ4Compressor;
+import org.apache.cassandra.io.sstable.CQLSSTableWriter;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.SSTableTombstoneWriter;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.locator.SimpleSnitch;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableId;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.security.EncryptionContext;
+import org.apache.cassandra.spark.cdc.CommitLog;
+import org.apache.cassandra.spark.cdc.CommitLogProvider;
+import org.apache.cassandra.spark.cdc.IPartitionUpdateWrapper;
+import org.apache.cassandra.spark.cdc.TableIdLookup;
+import org.apache.cassandra.spark.cdc.watermarker.Watermarker;
+import org.apache.cassandra.spark.data.CqlField;
+import org.apache.cassandra.spark.data.CqlTable;
+import org.apache.cassandra.spark.data.CqlType;
+import org.apache.cassandra.spark.data.ReplicationFactor;
+import org.apache.cassandra.spark.data.SSTablesSupplier;
+import org.apache.cassandra.spark.data.complex.CqlCollection;
+import org.apache.cassandra.spark.data.complex.CqlFrozen;
+import org.apache.cassandra.spark.data.complex.CqlList;
+import org.apache.cassandra.spark.data.complex.CqlMap;
+import org.apache.cassandra.spark.data.complex.CqlSet;
+import org.apache.cassandra.spark.data.complex.CqlTuple;
+import org.apache.cassandra.spark.data.complex.CqlUdt;
+import org.apache.cassandra.spark.data.partitioner.Partitioner;
+import org.apache.cassandra.spark.data.types.Ascii;
+import org.apache.cassandra.spark.data.types.BigInt;
+import org.apache.cassandra.spark.data.types.Blob;
+import org.apache.cassandra.spark.data.types.Boolean;
+import org.apache.cassandra.spark.data.types.Counter;
+import org.apache.cassandra.spark.data.types.Date;
+import org.apache.cassandra.spark.data.types.Decimal;
+import org.apache.cassandra.spark.data.types.Double;
+import org.apache.cassandra.spark.data.types.Duration;
+import org.apache.cassandra.spark.data.types.Empty;
+import org.apache.cassandra.spark.data.types.Float;
+import org.apache.cassandra.spark.data.types.Inet;
+import org.apache.cassandra.spark.data.types.Int;
+import org.apache.cassandra.spark.data.types.SmallInt;
+import org.apache.cassandra.spark.data.types.Text;
+import org.apache.cassandra.spark.data.types.Time;
+import org.apache.cassandra.spark.data.types.TimeUUID;
+import org.apache.cassandra.spark.data.types.Timestamp;
+import org.apache.cassandra.spark.data.types.TinyInt;
+import org.apache.cassandra.spark.data.types.VarChar;
+import org.apache.cassandra.spark.data.types.VarInt;
+import org.apache.cassandra.spark.reader.CdcScannerBuilder;
+import org.apache.cassandra.spark.reader.CompactionStreamScanner;
+import org.apache.cassandra.spark.reader.SchemaBuilder;
+import org.apache.cassandra.spark.reader.StreamScanner;
+import org.apache.cassandra.spark.sparksql.filters.CdcOffsetFilter;
+import org.apache.cassandra.spark.sparksql.filters.PartitionKeyFilter;
+import org.apache.cassandra.spark.sparksql.filters.PruneColumnFilter;
+import org.apache.cassandra.spark.sparksql.filters.SparkRangeFilter;
+import org.apache.cassandra.spark.stats.Stats;
+import org.apache.cassandra.spark.utils.ColumnTypes;
+import org.apache.cassandra.spark.utils.SparkClassLoaderOverride;
+import org.apache.cassandra.spark.utils.TimeProvider;
+import org.apache.cassandra.tools.JsonTransformer;
+import org.apache.cassandra.tools.Util;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.UUIDGen;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+@SuppressWarnings("unused")
+public class CassandraBridgeImplementation extends CassandraBridge
+{
+    private static volatile boolean setup = false;
+
+    private final Map<String, CqlField.NativeType> nativeTypes;
+    private final Map<Class<?>, Serializer<?>> kryoSerializers;
+
+    static
+    {
+        CassandraBridgeImplementation.setup();
+    }
+
+    public static synchronized void setup()
+    {
+        if (!CassandraBridgeImplementation.setup)
+        {
+            // We never want to enable mbean registration in the Cassandra code we use so disable it here
+            System.setProperty("org.apache.cassandra.disable_mbean_registration", "true");
+            Config.setClientMode(true);
+            // When we create a TableStreamScanner, we will set the partitioner directly on the table metadata
+            // using the supplied IIndexStreamScanner.Partitioner. CFMetaData::compile requires a partitioner to
+            // be set in DatabaseDescriptor before we can do that though, so we set one here in preparation.
+            DatabaseDescriptor.setPartitionerUnsafe(Murmur3Partitioner.instance);
+            DatabaseDescriptor.clientInitialization();
+            Config config = DatabaseDescriptor.getRawConfig();
+            config.memtable_flush_writers = 8;
+            config.diagnostic_events_enabled = false;
+            config.max_mutation_size_in_kb = config.commitlog_segment_size_in_mb * 1024 / 2;
+            config.concurrent_compactors = 4;
+            Path tempDirectory;
+            try
+            {
+                tempDirectory = Files.createTempDirectory(UUID.randomUUID().toString());
+            }
+            catch (IOException exception)
+            {
+                throw new RuntimeException(exception);
+            }
+            config.data_file_directories = new String[]{tempDirectory.toString()};
+            DatabaseDescriptor.setEndpointSnitch(new SimpleSnitch());
+            Keyspace.setInitialized();
+
+            setup = true;
+        }
+    }
+
+    public CassandraBridgeImplementation()
+    {
+        // Cassandra-version-specific Kryo serializers
+        kryoSerializers = new LinkedHashMap<>();
+        kryoSerializers.put(CqlField.class, new CqlField.Serializer(this));
+        kryoSerializers.put(CqlTable.class, new CqlTable.Serializer(this));
+        kryoSerializers.put(CqlUdt.class, new CqlUdt.Serializer(this));
+
+        nativeTypes = allTypes().stream().collect(Collectors.toMap(CqlField.CqlType::name, Function.identity()));
+    }
+
+    @Override
+    public AbstractMap.SimpleEntry<ByteBuffer, BigInteger> getPartitionKey(@NotNull CqlTable table,
+                                                                           @NotNull Partitioner partitioner,
+                                                                           @NotNull List<String> keys)
+    {
+        Preconditions.checkArgument(table.partitionKeys().size() > 0);
+        ByteBuffer partitionKey = buildPartitionKey(table, keys);
+        BigInteger partitionKeyTokenValue = hash(partitioner, partitionKey);
+        return new AbstractMap.SimpleEntry<>(partitionKey, partitionKeyTokenValue);
+    }
+
+    @VisibleForTesting
+    public static ByteBuffer buildPartitionKey(@NotNull CqlTable table, @NotNull List<String> keys)
+    {
+        List<AbstractType<?>> partitionKeyColumnTypes = partitionKeyColumnTypes(table);
+        if (table.partitionKeys().size() == 1)
+        {
+            // Single partition key
+            return partitionKeyColumnTypes.get(0).fromString(keys.get(0));
+        }
+        else
+        {
+            // Composite partition key
+            ByteBuffer[] buffers = new ByteBuffer[keys.size()];
+            for (int index = 0; index < buffers.length; index++)
+            {
+                buffers[index] = partitionKeyColumnTypes.get(index).fromString(keys.get(index));
+            }
+            return CompositeType.build(ByteBufferAccessor.instance, buffers);
+        }
+    }
+
+    @VisibleForTesting
+    public static List<AbstractType<?>> partitionKeyColumnTypes(@NotNull CqlTable table)
+    {
+        return table.partitionKeys().stream()
+                    .map(CqlField::type)
+                    .map(type -> (CqlType) type)
+                    .map(type -> type.dataType(true))
+                    .collect(Collectors.toList());
+    }
+
+    @Override
+    public TimeProvider timeProvider()
+    {
+        return FBUtilities::nowInSeconds;
+    }
+
+    @Override
+    public StreamScanner getCdcScanner(int partitionId,
+                                       @NotNull CqlTable table,
+                                       @NotNull Partitioner partitioner,
+                                       @NotNull CommitLogProvider commitLogProvider,
+                                       @NotNull TableIdLookup tableIdLookup,
+                                       @NotNull Stats stats,
+                                       @Nullable SparkRangeFilter sparkRangeFilter,
+                                       @Nullable CdcOffsetFilter offset,
+                                       int minimumReplicasPerMutation,
+                                       @NotNull Watermarker watermarker,
+                                       @NotNull String jobId,
+                                       @NotNull ExecutorService executorService,
+                                       @NotNull TimeProvider timeProvider)
+    {
+        // NOTE: Need to use SchemaBuilder to init keyspace if not already set in Cassandra schema instance
+        UUID tableId = tableIdLookup.lookup(table.keyspace(), table.table());
+        SchemaBuilder schemaBuilder = new SchemaBuilder(table, partitioner, tableId);
+        if (tableId != null)
+        {
+            // Verify TableMetadata and ColumnFamilyStore initialized in Schema
+            TableId tableIdAfter = TableId.fromUUID(tableId);
+            Preconditions.checkNotNull(Schema.instance.getTableMetadata(tableIdAfter),
+                                       "Table not initialized in the schema");
+            Preconditions.checkArgument(Objects.requireNonNull(Schema.instance.getKeyspaceInstance(table.keyspace()))
+                                               .hasColumnFamilyStore(tableIdAfter),
+                                        "ColumnFamilyStore not initialized in the schema");
+        }
+        TableMetadata metadata = schemaBuilder.tableMetaData();
+        return new CdcScannerBuilder(partitionId,
+                                     metadata,
+                                     partitioner,
+                                     commitLogProvider,
+                                     stats,
+                                     sparkRangeFilter,
+                                     offset,
+                                     minimumReplicasPerMutation,
+                                     watermarker,
+                                     jobId,
+                                     executorService,
+                                     timeProvider).build();
+    }
+
+    @Override
+    public StreamScanner getCompactionScanner(@NotNull CqlTable table,
+                                              @NotNull Partitioner partitioner,
+                                              @NotNull SSTablesSupplier ssTables,
+                                              @Nullable SparkRangeFilter sparkRangeFilter,
+                                              @NotNull Collection<PartitionKeyFilter> partitionKeyFilters,
+                                              @Nullable PruneColumnFilter columnFilter,
+                                              @NotNull TimeProvider timeProvider,
+                                              boolean readIndexOffset,
+                                              boolean useIncrementalRepair,
+                                              @NotNull Stats stats)
+    {
+        // NOTE: Need to use SchemaBuilder to init keyspace if not already set in Cassandra Schema instance
+        SchemaBuilder schemaBuilder = new SchemaBuilder(table, partitioner);
+        TableMetadata metadata = schemaBuilder.tableMetaData();
+        return new CompactionStreamScanner(metadata, partitioner, timeProvider, ssTables.openAll((ssTable, isRepairPrimary) ->
+                org.apache.cassandra.spark.reader.SSTableReader.builder(metadata, ssTable)
+                                                               .withSparkRangeFilter(sparkRangeFilter)
+                                                               .withPartitionKeyFilters(partitionKeyFilters)
+                                                               .withColumnFilter(columnFilter)
+                                                               .withReadIndexOffset(readIndexOffset)
+                                                               .withStats(stats)
+                                                               .useIncrementalRepair(useIncrementalRepair)
+                                                               .isRepairPrimary(isRepairPrimary)
+                                                               .build()));
+    }
+
+    @Override
+    public CassandraVersion getVersion()
+    {
+        return CassandraVersion.FOURZERO;
+    }
+
+    @Override
+    public BigInteger hash(Partitioner partitioner, ByteBuffer key)
+    {
+        switch (partitioner)
+        {
+            case RandomPartitioner:
+                return RandomPartitioner.instance.getToken(key).getTokenValue();
+            case Murmur3Partitioner:
+                return BigInteger.valueOf((long) Murmur3Partitioner.instance.getToken(key).getTokenValue());
+            default:
+                throw new UnsupportedOperationException("Unexpected partitioner: " + partitioner);
+        }
+    }
+
+    @Override
+    public UUID getTimeUUID()
+    {
+        return UUIDGen.getTimeUUID();
+    }
+
+    @Override
+    public CqlTable buildSchema(String createStatement,
+                                String keyspace,
+                                ReplicationFactor replicationFactor,
+                                Partitioner partitioner,
+                                Set<String> udts,
+                                @Nullable UUID tableId,
+                                int indexCount)
+    {
+        return new SchemaBuilder(createStatement, keyspace, replicationFactor, partitioner, udts, tableId, indexCount).build();
+    }
+
+    // CQL Type Parser
+
+    @Override
+    public Map<String, ? extends CqlField.NativeType> nativeTypeNames()
+    {
+        return nativeTypes;
+    }
+
+    @Override
+    public CqlField.CqlType readType(CqlField.CqlType.InternalType type, Input input)
+    {
+        switch (type)
+        {
+            case NativeCql:
+                return nativeType(input.readString());
+            case Set:
+            case List:
+            case Map:
+            case Tuple:
+                return CqlCollection.read(type, input, this);
+            case Frozen:
+                return CqlFrozen.build(CqlField.CqlType.read(input, this));
+            case Udt:
+                return CqlUdt.read(input, this);
+            default:
+                throw new IllegalStateException("Unknown CQL type, cannot deserialize");
+        }
+    }
+
+    @Override
+    public Ascii ascii()
+    {
+        return Ascii.INSTANCE;
+    }
+
+    @Override
+    public Blob blob()
+    {
+        return Blob.INSTANCE;
+    }
+
+    @Override
+    public Boolean bool()
+    {
+        return Boolean.INSTANCE;
+    }
+
+    @Override
+    public Counter counter()
+    {
+        return Counter.INSTANCE;
+    }
+
+    @Override
+    public BigInt bigint()
+    {
+        return BigInt.INSTANCE;
+    }
+
+    @Override
+    public Date date()
+    {
+        return Date.INSTANCE;
+    }
+
+    @Override
+    public Decimal decimal()
+    {
+        return Decimal.INSTANCE;
+    }
+
+    @Override
+    public Double aDouble()
+    {
+        return Double.INSTANCE;
+    }
+
+    @Override
+    public Duration duration()
+    {
+        return Duration.INSTANCE;
+    }
+
+    @Override
+    public Empty empty()
+    {
+        return Empty.INSTANCE;
+    }
+
+    @Override
+    public Float aFloat()
+    {
+        return Float.INSTANCE;
+    }
+
+    @Override
+    public Inet inet()
+    {
+        return Inet.INSTANCE;
+    }
+
+    @Override
+    public Int aInt()
+    {
+        return Int.INSTANCE;
+    }
+
+    @Override
+    public SmallInt smallint()
+    {
+        return SmallInt.INSTANCE;
+    }
+
+    @Override
+    public Text text()
+    {
+        return Text.INSTANCE;
+    }
+
+    @Override
+    public Time time()
+    {
+        return Time.INSTANCE;
+    }
+
+    @Override
+    public Timestamp timestamp()
+    {
+        return Timestamp.INSTANCE;
+    }
+
+    @Override
+    public TimeUUID timeuuid()
+    {
+        return TimeUUID.INSTANCE;
+    }
+
+    @Override
+    public TinyInt tinyint()
+    {
+        return TinyInt.INSTANCE;
+    }
+
+    @Override
+    public org.apache.cassandra.spark.data.types.UUID uuid()
+    {
+        return org.apache.cassandra.spark.data.types.UUID.INSTANCE;
+    }
+
+    @Override
+    public VarChar varchar()
+    {
+        return VarChar.INSTANCE;
+    }
+
+    @Override
+    public VarInt varint()
+    {
+        return VarInt.INSTANCE;
+    }
+
+    @Override
+    public CqlField.CqlType collection(String name, CqlField.CqlType... types)
+    {
+        return CqlCollection.build(name, types);
+    }
+
+    @Override
+    public CqlList list(CqlField.CqlType type)
+    {
+        return CqlCollection.list(type);
+    }
+
+    @Override
+    public CqlSet set(CqlField.CqlType type)
+    {
+        return CqlCollection.set(type);
+    }
+
+    @Override
+    public CqlMap map(CqlField.CqlType keyType, CqlField.CqlType valueType)
+    {
+        return CqlCollection.map(keyType, valueType);
+    }
+
+    @Override
+    public CqlTuple tuple(CqlField.CqlType... types)
+    {
+        return CqlCollection.tuple(types);
+    }
+
+    @Override
+    public CqlField.CqlType frozen(CqlField.CqlType type)
+    {
+        return CqlFrozen.build(type);
+    }
+
+    @Override
+    public CqlField.CqlUdtBuilder udt(String keyspace, String name)
+    {
+        return CqlUdt.builder(keyspace, name);
+    }
+
+    @Override
+    public synchronized void writeSSTable(Partitioner partitioner,
+                                          String keyspace,
+                                          String table,
+                                          Path directory,
+                                          String createStatement,
+                                          String insertStatement,
+                                          String updateStatement,
+                                          boolean upsert,
+                                          Set<CqlField.CqlUdt> udts,
+                                          Consumer<Writer> writer)
+    {
+        CQLSSTableWriter.Builder builder = CQLSSTableWriter.builder()
+                                                                 .inDirectory(directory.toFile())
+                                                                 .forTable(createStatement)
+                                                                 .withPartitioner(getPartitioner(partitioner))
+                                                                 .using(upsert ? updateStatement : insertStatement)
+                                                                 .withBufferSizeInMB(128);
+
+        for (CqlField.CqlUdt udt : udts)
+        {
+            // Add user-defined types to CQL writer
+            builder.withType(udt.createStatement(keyspace));
+        }
+
+        // TODO: Remove me once CQLSSTableWriter.Builder synchronize on schema (see CASSANDRA-TBD)
+        //       build update schema, we need to synchornize
+        try (CQLSSTableWriter ssTable = CassandraSchema.apply(s -> builder.build()))
+        {
+            writer.accept(values -> {
+                try
+                {
+                    ssTable.addRow(values);
+                }
+                catch (IOException exception)
+                {
+                    throw new RuntimeException(exception);
+                }
+            });
+        }
+        catch (IOException exception)
+        {
+            throw new RuntimeException(exception);
+        }
+    }
+
+    public static IPartitioner getPartitioner(Partitioner partitioner)
+    {
+        return partitioner == Partitioner.Murmur3Partitioner ? Murmur3Partitioner.instance : RandomPartitioner.instance;
+    }
+
+    @Override
+    public SSTableWriter getSSTableWriter(String inDirectory,
+                                          String partitioner,
+                                          String createStatement,
+                                          String insertStatement,
+                                          boolean isSorted,
+                                          int bufferSizeMB)
+    {
+        return new SSTableWriterImplementation(inDirectory, partitioner, createStatement, insertStatement, isSorted, bufferSizeMB);
+    }
+
+    // CDC Configuration
+
+    @Override
+    public void setCDC(Path path)
+    {
+        DatabaseDescriptor.getRawConfig().cdc_raw_directory = path + "/cdc";
+        DatabaseDescriptor.setCDCEnabled(true);
+        DatabaseDescriptor.setCommitLogSync(Config.CommitLogSync.periodic);
+        DatabaseDescriptor.setCommitLogCompression(new ParameterizedClass("LZ4Compressor", Collections.emptyMap()));
+        DatabaseDescriptor.setEncryptionContext(new EncryptionContext());
+        DatabaseDescriptor.setCommitLogSyncPeriod(30);
+        DatabaseDescriptor.setCommitLogMaxCompressionBuffersPerPool(3);
+        DatabaseDescriptor.setCommitLogSyncGroupWindow(30);
+        DatabaseDescriptor.setCommitLogSegmentSize(32);
+        DatabaseDescriptor.getRawConfig().commitlog_total_space_in_mb = 1024;
+        DatabaseDescriptor.setCommitLogSegmentMgrProvider(commitLog -> new CommitLogSegmentManagerCDC(commitLog, path + "/commitlog"));
+    }
+
+    @Override
+    public void setCommitLogPath(Path path)
+    {
+        DatabaseDescriptor.getRawConfig().commitlog_directory = path + "/commitlog";
+        DatabaseDescriptor.getRawConfig().hints_directory = path + "/hints";
+        DatabaseDescriptor.getRawConfig().saved_caches_directory = path + "/saved_caches";
+    }
+
+    @Override
+    @VisibleForTesting
+    public ICommitLog testCommitLog(File folder)
+    {
+        return new TestCommitLog(folder);
+    }
+
+    // CommitLog
+
+    @VisibleForTesting
+    public static final class MutationWrapper implements IMutation
+    {
+        public final Mutation mutation;
+
+        private MutationWrapper(Mutation mutation)
+        {
+            this.mutation = mutation;
+        }
+
+        static MutationWrapper wrap(Mutation mutation)
+        {
+            return new MutationWrapper(mutation);
+        }
+    }
+
+    @Override
+    @VisibleForTesting
+    public void log(CqlTable table, ICommitLog log, IRow row, long timestamp)
+    {
+        Mutation mutation = makeMutation(table, row, timestamp);
+        log.add(MutationWrapper.wrap(mutation));
+    }
+
+    // CHECKSTYLE IGNORE: Long method
+    @NotNull
+    @VisibleForTesting
+    private Mutation makeMutation(CqlTable cqlTable, IRow row, long timestamp)
+    {
+        TableMetadata table = Schema.instance.getTableMetadata(cqlTable.keyspace(), cqlTable.table());
+        assert table != null;
+
+        Row.Builder rowBuilder = BTreeRow.sortedBuilder();
+        if (row.isInsert())
+        {
+            rowBuilder.addPrimaryKeyLivenessInfo(LivenessInfo.create(timestamp, timeProvider().nowInTruncatedSeconds()));
+        }
+        Row staticRow = Rows.EMPTY_STATIC_ROW;
+
+        // Build partition key
+        List<CqlField> partitionKeys = cqlTable.partitionKeys();
+        ByteBuffer partitionKey = ColumnTypes.buildPartitionKey(partitionKeys,
+                                                                partitionKeys.stream()
+                                                                             .map(field -> row.get(field.position()))
+                                                                             .toArray());
+
+        DecoratedKey decoratedPartitionKey = table.partitioner.decorateKey(partitionKey);
+
+        // Create a mutation and return early
+        if (isPartitionDeletion(cqlTable, row))
+        {
+            PartitionUpdate delete = PartitionUpdate.fullPartitionDelete(table, partitionKey, timestamp, timeProvider().nowInTruncatedSeconds());
+            return new Mutation(delete);
+        }
+
+        List<CqlField> clusteringKeys = cqlTable.clusteringKeys();
+
+        // Create a mutation with range tombstones
+        if (row.rangeTombstones() != null && !row.rangeTombstones().isEmpty())
+        {
+            PartitionUpdate.SimpleBuilder updateBuilder = PartitionUpdate.simpleBuilder(table, decoratedPartitionKey)
+                                                                         .timestamp(timestamp)
+                                                                         .nowInSec(timeProvider().nowInTruncatedSeconds());
+            for (RangeTombstone rangeTombstone : row.rangeTombstones())
+            {
+                // Range tombstone builder is built when partition update builder builds
+                PartitionUpdate.SimpleBuilder.RangeTombstoneBuilder tombstoneBuilder = updateBuilder.addRangeTombstone();
+                // Returns the same ref. just to make compiler happy
+                tombstoneBuilder = rangeTombstone.open.inclusive ? tombstoneBuilder.inclStart() : tombstoneBuilder.exclStart();
+                Object[] startValues = clusteringKeys.stream()
+                                                     .map(field -> {
+                                                         Object value = rangeTombstone.open.values[field.position() - cqlTable.numPartitionKeys()];
+                                                         return value != null ? field.serialize(value) : null;
+                                                     })
+                                                     .filter(Objects::nonNull)
+                                                     .toArray(ByteBuffer[]::new);
+                tombstoneBuilder.start(startValues);
+                tombstoneBuilder = rangeTombstone.close.inclusive ? tombstoneBuilder.inclEnd() : tombstoneBuilder.exclEnd();
+                Object[] endValues = clusteringKeys.stream()
+                                                   .map(field -> {
+                                                       Object value = rangeTombstone.close.values[field.position() - cqlTable.numPartitionKeys()];
+                                                       return value != null ? field.serialize(value) : null;
+                                                   })
+                                                   .filter(Objects::nonNull)
+                                                   .toArray(ByteBuffer[]::new);
+                tombstoneBuilder.end(endValues);
+            }
+            return new Mutation(updateBuilder.build());
+        }
+
+        // Build clustering key
+        if (!clusteringKeys.isEmpty())
+        {
+            rowBuilder.newRow(Clustering.make(clusteringKeys.stream()
+                                                            .map(field -> field.serialize(row.get(field.position())))
+                                                            .toArray(ByteBuffer[]::new)));
+        }
+        else
+        {
+            rowBuilder.newRow(Clustering.EMPTY);
+        }
+
+        if (!row.isDeleted())
+        {
+            BiConsumer<Row.Builder, CqlField> rowBuildFunc = (builder, field) -> {
+                CqlType type = (CqlType) field.type();
+                ColumnMetadata column = table.getColumn(new ColumnIdentifier(field.name(), false));
+                Object value = row.get(field.position());
+                if (value == CassandraBridge.UNSET_MARKER)
+                {
+                    // CHECKSTYLE IGNORE: Do not add the cell, a.k.a. unset
+                }
+                else if (value == null)
+                {
+                    if (column.isComplex())
+                    {
+                        type.addComplexTombstone(builder, column, timestamp);
+                    }
+                    else
+                    {
+                        type.addTombstone(builder, column, timestamp);
+                    }
+                }
+                else if (value instanceof CollectionElement)
+                {
+                    CollectionElement element = (CollectionElement) value;
+                    if (element.value == null)
+                    {
+                        type.addTombstone(builder, column, timestamp, element.cellPath);
+                    }
+                    else
+                    {
+                        type.addCell(builder, column, timestamp, element.value, element.cellPath);
+                    }
+                }
+                else
+                {
+                    type.addCell(builder, column, timestamp, value);
+                }
+            };
+
+            if (!cqlTable.staticColumns().isEmpty())
+            {
+                Row.Builder staticRowBuilder = BTreeRow.sortedBuilder();
+                staticRowBuilder.newRow(Clustering.STATIC_CLUSTERING);
+                for (CqlField field : cqlTable.staticColumns())
+                {
+                    rowBuildFunc.accept(staticRowBuilder, field);
+                }
+                staticRow = staticRowBuilder.build();  // Replace the empty row with the new static row built
+            }
+
+            // Build value cells
+            for (CqlField field : cqlTable.valueColumns())
+            {
+                rowBuildFunc.accept(rowBuilder, field);
+            }
+        }
+        else
+        {
+            rowBuilder.addRowDeletion(Row.Deletion.regular(new DeletionTime(timestamp, timeProvider().nowInTruncatedSeconds())));
+        }
+
+        return new Mutation(PartitionUpdate.singleRowUpdate(table, decoratedPartitionKey, rowBuilder.build(), staticRow));
+    }
+
+    @Override
+    @VisibleForTesting
+    protected boolean isPartitionDeletion(CqlTable table, IRow row)
+    {
+        List<CqlField> clusteringKeys = table.clusteringKeys();
+        List<CqlField> valueFields = table.valueColumns();
+        List<CqlField> staticFields = table.staticColumns();
+        for (CqlField field : Iterables.concat(clusteringKeys, valueFields, staticFields))
+        {
+            if (row.get(field.position()) != null)
+            {
+                return false;
+            }
+        }
+        return true;
+    }
+
+    @Override
+    @VisibleForTesting
+    protected boolean isRowDeletion(CqlTable table, IRow row)
+    {
+        return row.isDeleted();
+    }
+
+    @Override
+    @VisibleForTesting
+    public Object livingCollectionElement(ByteBuffer cellPath, Object value)
+    {
+        return CollectionElement.living(CellPath.create(cellPath), value);
+    }
+
+    @Override
+    @VisibleForTesting
+    public Object deletedCollectionElement(ByteBuffer cellPath)
+    {
+        return CollectionElement.deleted(CellPath.create(cellPath));
+    }
+
+    @Override
+    @VisibleForTesting
+    public Set<Long> readLog(CqlTable table, CommitLog log, Watermarker watermarker)
+    {
+        SchemaBuilder schemaBuilder = new SchemaBuilder(table, Partitioner.Murmur3Partitioner);
+        TableMetadata metadata = schemaBuilder.tableMetaData();
+        try (BufferingCommitLogReader reader = new BufferingCommitLogReader(metadata, log, watermarker))
+        {
+            List<PartitionUpdateWrapper> updates = reader.result().updates();
+            Set<Long> keys = updates.stream()
+                                          .map(update -> update.partitionKey().getKey().getLong())
+                                          .collect(Collectors.toSet());
+            assert updates.size() == keys.size() : "Duplicate keys have been read from the commit log";
+            return keys;
+        }
+    }
+
+    // Version-Specific Test Utility Methods
+
+    @Override
+    @VisibleForTesting
+    public void writeTombstoneSSTable(Partitioner partitioner,
+                                      Path directory,
+                                      String createStatement,
+                                      String deleteStatement,
+                                      Consumer<Writer> consumer)
+    {
+        try (SSTableTombstoneWriter writer = SSTableTombstoneWriter.builder()
+                                                                         .inDirectory(directory.toFile())
+                                                                         .forTable(createStatement)
+                                                                         .withPartitioner(getPartitioner(partitioner))
+                                                                         .using(deleteStatement)
+                                                                         .withBufferSizeInMB(128)
+                                                                         .build())
+        {
+            consumer.accept(values -> {
+                try
+                {
+                    writer.addRow(values);
+                }
+                catch (IOException exception)
+                {
+                    throw new RuntimeException(exception);
+                }
+            });
+        }
+        catch (IOException exception)
+        {
+            throw new RuntimeException(exception);
+        }
+    }
+
+    @Override
+    @VisibleForTesting
+    public void sstableToJson(Path dataDbFile, OutputStream output) throws FileNotFoundException
+    {
+        if (!Files.exists(dataDbFile))
+        {
+            throw new FileNotFoundException("Cannot find file " + dataDbFile.toAbsolutePath());
+        }
+        if (!Descriptor.isValidFile(dataDbFile.toFile()))
+        {
+            throw new RuntimeException("Invalid sstable file");
+        }
+
+        Descriptor desc = Descriptor.fromFilename(dataDbFile.toAbsolutePath().toString());
+        try
+        {
+            TableMetadataRef metadata = TableMetadataRef.forOfflineTools(Util.metadataFromSSTable(desc));
+            SSTableReader ssTable = SSTableReader.openNoValidation(desc, metadata);
+            ISSTableScanner currentScanner = ssTable.getScanner();
+            Stream<UnfilteredRowIterator> partitions = Util.iterToStream(currentScanner);
+            JsonTransformer.toJson(currentScanner, partitions, false, metadata.get(), output);
+        }
+        catch (IOException exception)
+        {
+            throw new RuntimeException(exception);
+        }
+    }
+
+    @Override
+    @VisibleForTesting
+    public Object toTupleValue(CqlField.CqlTuple type, Object[] values)
+    {
+        return CqlTuple.toTupleValue(getVersion(), (CqlTuple) type, values);
+    }
+
+    @Override
+    @VisibleForTesting
+    public Object toUserTypeValue(CqlField.CqlUdt type, Map<String, Object> values)
+    {
+        return CqlUdt.toUserTypeValue(getVersion(), (CqlUdt) type, values);
+    }
+
+    // Compression Utils
+
+    private static final ICompressor COMPRESSOR = LZ4Compressor.create(Collections.emptyMap());
+
+    @Override
+    public ByteBuffer compress(byte[] bytes) throws IOException
+    {
+        ByteBuffer input = COMPRESSOR.preferredBufferType().allocate(bytes.length);
+        input.put(bytes);
+        input.flip();
+        return compress(input);
+    }
+
+    @Override
+    public ByteBuffer compress(ByteBuffer input) throws IOException
+    {
+        int length = input.remaining();  // Store uncompressed length as 4 byte int
+        // 4 extra bytes to store uncompressed length
+        ByteBuffer output = COMPRESSOR.preferredBufferType().allocate(4 + COMPRESSOR.initialCompressedBufferLength(length));
+        output.putInt(length);
+        COMPRESSOR.compress(input, output);
+        output.flip();
+        return output;
+    }
+
+    @Override
+    public ByteBuffer uncompress(byte[] bytes) throws IOException
+    {
+        ByteBuffer input = COMPRESSOR.preferredBufferType().allocate(bytes.length);
+        input.put(bytes);
+        input.flip();
+        return uncompress(input);
+    }
+
+    @Override
+    public ByteBuffer uncompress(ByteBuffer input) throws IOException
+    {
+        ByteBuffer output = COMPRESSOR.preferredBufferType().allocate(input.getInt());
+        COMPRESSOR.uncompress(input, output);
+        output.flip();
+        return output;
+    }
+
+    // Kryo Serializers
+
+    @Override
+    public Serializer<? extends IPartitionUpdateWrapper> getPartitionUpdateSerializer(String keyspace,
+                                                                                      String table,
+                                                                                      boolean includePartitionUpdate)
+    {
+        return new PartitionUpdateWrapper.Serializer(keyspace, table, includePartitionUpdate);
+    }
+
+    // Kryo/Java (De-)Serialization
+
+    @Override
+    public void kryoRegister(Kryo kryo)
+    {
+        kryoSerializers.forEach(kryo::register);
+    }
+
+    @Override
+    public void javaSerialize(ObjectOutputStream out, Serializable object)
+    {
+        try
+        {
+            out.writeObject(object);
+        }
+        catch (IOException exception)
+        {
+            throw new RuntimeException(exception);
+        }
+    }
+
+    @Override
+    public <T> T javaDeserialize(ObjectInputStream in, Class<T> type)
+    {
+        try (SparkClassLoaderOverride override = new SparkClassLoaderOverride(in, getClass().getClassLoader()))
+        {
+            return type.cast(in.readObject());
+        }
+        catch (IOException | ClassNotFoundException exception)
+        {
+            throw new RuntimeException(exception);
+        }
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/bridge/CassandraSchema.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/bridge/CassandraSchema.java
new file mode 100644
index 0000000..b6623ff
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/bridge/CassandraSchema.java
@@ -0,0 +1,61 @@
+/*
+ * 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.cassandra.bridge;
+
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+import org.apache.cassandra.schema.Schema;
+
+public final class CassandraSchema
+{
+    private CassandraSchema()
+    {
+        throw new IllegalStateException("Do not instantiate!");
+    }
+
+    /**
+     * Update cassandra schema with synchronization
+     *
+     * @param updater updates schema
+     */
+    public static void update(Consumer<Schema> updater)
+    {
+        synchronized (Schema.instance)
+        {
+            updater.accept(Schema.instance);
+        }
+    }
+
+    /**
+     * Update cassandra schema and return a result with synchronization
+     *
+     * @param <T> type of the returned value
+     * @param updater updates schema and return a result
+     * @return a new value depending on the updater
+     */
+    public static <T> T apply(Function<Schema, T> updater)
+    {
+        synchronized (Schema.instance)
+        {
+            return updater.apply(Schema.instance);
+        }
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/bridge/CollectionElement.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/bridge/CollectionElement.java
new file mode 100644
index 0000000..333cce4
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/bridge/CollectionElement.java
@@ -0,0 +1,46 @@
+/*
+ * 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.cassandra.bridge;
+
+import org.apache.cassandra.db.rows.CellPath;
+
+public final class CollectionElement
+{
+    // The path to store the value in the collection, consider it as the key
+    public final CellPath cellPath;
+    // The value to be stored in the collection
+    public final Object value;
+
+    private CollectionElement(CellPath cellPath, Object value)
+    {
+        this.cellPath = cellPath;
+        this.value = value;
+    }
+
+    public static CollectionElement living(CellPath cellPath, Object value)
+    {
+        return new CollectionElement(cellPath, value);
+    }
+
+    public static CollectionElement deleted(CellPath cellPath)
+    {
+        return new CollectionElement(cellPath, null);
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/bridge/SSTableWriterImplementation.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/bridge/SSTableWriterImplementation.java
new file mode 100644
index 0000000..c53fc25
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/bridge/SSTableWriterImplementation.java
@@ -0,0 +1,82 @@
+/*
+ * 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.cassandra.bridge;
+
+import java.io.IOException;
+
+import org.apache.cassandra.config.Config;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.dht.RandomPartitioner;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.io.sstable.CQLSSTableWriter;
+
+public class SSTableWriterImplementation implements SSTableWriter
+{
+    static
+    {
+        Config.setClientMode(true);
+    }
+
+    private final CQLSSTableWriter writer;
+
+    public SSTableWriterImplementation(String inDirectory,
+                                       String partitioner,
+                                       String createStatement,
+                                       String insertStatement,
+                                       boolean isSorted,
+                                       int bufferSizeMB)
+    {
+        IPartitioner cassPartitioner = partitioner.toLowerCase().contains("random") ? new RandomPartitioner()
+                                                                                    : new Murmur3Partitioner();
+        CQLSSTableWriter.Builder builder = CQLSSTableWriter.builder()
+                                                           .inDirectory(inDirectory)
+                                                           .forTable(createStatement)
+                                                           .withPartitioner(cassPartitioner)
+                                                           .using(insertStatement)
+                                                           .withBufferSizeInMB(bufferSizeMB);
+        if (isSorted)
+        {
+            builder.sorted();
+        }
+        // TODO: Remove me once CQLSSTableWriter.Builder synchronize on schema (see CASSANDRA-TBD)
+        //       build update schema, we need to synchronize
+        writer = CassandraSchema.apply(s -> builder.build());
+    }
+
+    @Override
+    public void addRow(Object... values) throws IOException
+    {
+        try
+        {
+            writer.addRow(values);
+        }
+        catch (InvalidRequestException exception)
+        {
+            throw new RuntimeException(exception);
+        }
+    }
+
+    @Override
+    public void close() throws IOException
+    {
+        writer.close();
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/bridge/TestCommitLog.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/bridge/TestCommitLog.java
new file mode 100644
index 0000000..8ff4370
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/bridge/TestCommitLog.java
@@ -0,0 +1,92 @@
+/*
+ * 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.cassandra.bridge;
+
+import java.io.File;
+import java.io.IOException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.commitlog.CommitLog;
+import org.apache.cassandra.spark.utils.IOUtils;
+
+public class TestCommitLog implements CassandraBridge.ICommitLog
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(TestCommitLog.class);
+
+    private final File folder;
+
+    public TestCommitLog(File folder)
+    {
+        this.folder = folder;
+        start();
+    }
+
+    @Override
+    public synchronized void start()
+    {
+        LOGGER.info("Starting CommitLog");
+        CommitLog.instance.start();
+    }
+
+    @Override
+    public synchronized void stop()
+    {
+        try
+        {
+            LOGGER.info("Shutting down CommitLog");
+            CommitLog.instance.shutdownBlocking();
+        }
+        catch (InterruptedException exception)
+        {
+            Thread.currentThread().interrupt();
+            throw new RuntimeException(exception);
+        }
+    }
+
+    @Override
+    public synchronized void clear()
+    {
+        IOUtils.clearDirectory(folder.toPath(), path -> LOGGER.info("Deleting CommitLog: " + path.toString()));
+    }
+
+    @Override
+    public void add(CassandraBridge.IMutation mutation)
+    {
+        if (mutation instanceof CassandraBridgeImplementation.MutationWrapper)
+        {
+            CommitLog.instance.add(((CassandraBridgeImplementation.MutationWrapper) mutation).mutation);
+        }
+    }
+
+    @Override
+    public void sync()
+    {
+        try
+        {
+            CommitLog.instance.sync(true);
+        }
+        catch (IOException exception)
+        {
+            throw new RuntimeException(exception);
+        }
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/cql3/functions/types/TupleHelper.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/cql3/functions/types/TupleHelper.java
new file mode 100644
index 0000000..cbffd69
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/cql3/functions/types/TupleHelper.java
@@ -0,0 +1,58 @@
+/*
+ * 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.cassandra.cql3.functions.types;
+
+import java.util.stream.Collectors;
+
+import org.apache.cassandra.spark.data.CqlType;
+import org.apache.cassandra.spark.data.complex.CqlTuple;
+import org.apache.cassandra.transport.ProtocolVersion;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Helper methods to access package-private Tuple methods
+ */
+public final class TupleHelper
+{
+    private TupleHelper()
+    {
+        throw new IllegalStateException(getClass() + " is static utility class and shall not be instantiated");
+    }
+
+    @NotNull
+    public static TupleType buildTupleType(CqlTuple tuple, boolean isFrozen)
+    {
+        return new TupleType(tuple.types().stream()
+                .map(type -> ((CqlType) type).driverDataType(isFrozen))
+                .collect(Collectors.toList()), ProtocolVersion.V3, CqlType.CODEC_REGISTRY);
+    }
+
+    @NotNull
+    public static TupleValue buildTupleValue(CqlTuple tuple)
+    {
+        return buildTupleValue(tuple, false);
+    }
+
+    @NotNull
+    public static TupleValue buildTupleValue(CqlTuple tuple, boolean isFrozen)
+    {
+        return new TupleValue(buildTupleType(tuple, isFrozen));
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/cql3/functions/types/UserTypeHelper.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/cql3/functions/types/UserTypeHelper.java
new file mode 100644
index 0000000..75f9aac
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/cql3/functions/types/UserTypeHelper.java
@@ -0,0 +1,59 @@
+/*
+ * 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.cassandra.cql3.functions.types;
+
+import java.util.Collection;
+
+import org.apache.cassandra.spark.data.CqlType;
+import org.apache.cassandra.transport.ProtocolVersion;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Helper methods to access package-private UDT methods
+ */
+public final class UserTypeHelper
+{
+    private UserTypeHelper()
+    {
+        throw new IllegalStateException(getClass() + " is static utility class and shall not be instantiated");
+    }
+
+    @NotNull
+    public static UDTValue newUDTValue(UserType userType)
+    {
+        return new UDTValue(userType);
+    }
+
+    @NotNull
+    public static UserType newUserType(String keyspace,
+                                       String typeName,
+                                       boolean frozen,
+                                       Collection<UserType.Field> fields,
+                                       ProtocolVersion protocolVersion)
+    {
+        return new UserType(keyspace, typeName, frozen, fields, protocolVersion, CqlType.CODEC_REGISTRY);
+    }
+
+    @NotNull
+    public static UserType.Field newField(String name, DataType type)
+    {
+        return new UserType.Field(name, type);
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/db/commitlog/BufferingCommitLogReader.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/db/commitlog/BufferingCommitLogReader.java
new file mode 100644
index 0000000..31d2a3b
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/db/commitlog/BufferingCommitLogReader.java
@@ -0,0 +1,747 @@
+/*
+ * 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.cassandra.db.commitlog;
+
+import java.io.DataOutputStream;
+import java.io.EOFException;
+import java.io.IOException;
+import java.math.BigInteger;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.zip.CRC32;
+import javax.annotation.concurrent.NotThreadSafe;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.Mutation;
+import org.apache.cassandra.db.partitions.AbstractBTreePartition;
+import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.db.rows.DeserializationHelper;
+import org.apache.cassandra.exceptions.UnknownTableException;
+import org.apache.cassandra.io.util.CdcRandomAccessReader;
+import org.apache.cassandra.io.util.DataInputBuffer;
+import org.apache.cassandra.io.util.FileDataInput;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.io.util.RebufferingInputStream;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.spark.cdc.watermarker.Watermarker;
+import org.apache.cassandra.spark.exceptions.TransportFailureException;
+import org.apache.cassandra.spark.reader.ReaderUtils;
+import org.apache.cassandra.spark.sparksql.filters.CdcOffsetFilter;
+import org.apache.cassandra.spark.sparksql.filters.SparkRangeFilter;
+import org.apache.cassandra.spark.utils.LoggerHelper;
+import org.apache.cassandra.spark.utils.ThrowableUtils;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.Pair;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.cassandra.utils.FBUtilities.updateChecksumInt;
+
+/**
+ * Copied and refactored from org.apache.cassandra.db.commitlog.CommitLogReader
+ * to read from generic source not tied to java.io.File and local file system
+ */
+@NotThreadSafe
+public class BufferingCommitLogReader implements CommitLogReadHandler, AutoCloseable, Comparable<BufferingCommitLogReader>
+{
+    private static final int LEGACY_END_OF_SEGMENT_MARKER = 0;
+
+    @VisibleForTesting
+    public static final int ALL_MUTATIONS = -1;
+    private final TableMetadata table;
+    private final org.apache.cassandra.spark.cdc.CommitLog log;
+    @Nullable
+    final CdcOffsetFilter offsetFilter;
+    private final CRC32 checksum;
+    List<PartitionUpdateWrapper> updates;
+    @Nullable
+    private final SparkRangeFilter sparkRangeFilter;
+    private byte[] buffer;
+
+    private RandomAccessReader reader;
+    private CommitLogDescriptor descriptor = null;
+    private final ReadStatusTracker statusTracker;
+    private int position = 0;
+
+    @NotNull
+    private final org.apache.cassandra.spark.cdc.CommitLog.Marker highWaterMark;
+
+    private final LoggerHelper logger;
+
+    @VisibleForTesting
+    public BufferingCommitLogReader(@NotNull TableMetadata table,
+                                    @NotNull org.apache.cassandra.spark.cdc.CommitLog log,
+                                    @NotNull Watermarker watermarker)
+    {
+        this(table, null, log, null, watermarker.highWaterMark(log.instance()), 0);
+    }
+
+    public BufferingCommitLogReader(@NotNull TableMetadata table,
+                                    @Nullable CdcOffsetFilter offsetFilter,
+                                    @NotNull org.apache.cassandra.spark.cdc.CommitLog log,
+                                    @Nullable SparkRangeFilter sparkRangeFilter,
+                                    @Nullable org.apache.cassandra.spark.cdc.CommitLog.Marker highWaterMark,
+                                    int partitionId)
+    {
+        this.table = table;
+        this.offsetFilter = offsetFilter;
+        this.log = log;
+        this.updates = new ArrayList<>();
+        this.sparkRangeFilter = sparkRangeFilter;
+        this.statusTracker = new ReadStatusTracker(ALL_MUTATIONS, false);
+        this.checksum = new CRC32();
+        this.buffer = new byte[CdcRandomAccessReader.DEFAULT_BUFFER_SIZE];
+        this.reader = BufferingCommitLogReader.reader(log);
+        this.logger = new LoggerHelper(LoggerFactory.getLogger(BufferingCommitLogReader.class),
+                                       "instance", log.instance().nodeName(),
+                                       "dc", log.instance().dataCenter(),
+                                       "log", log.name(),
+                                       "size", log.maxOffset(),
+                                       "partitionId", partitionId);
+
+        this.highWaterMark = highWaterMark != null ? highWaterMark : log.zeroMarker();
+
+        try
+        {
+            readHeader();
+            if (skip())
+            {
+                // If we can skip this CommitLog, close immediately
+                close();
+            }
+            else
+            {
+                read();
+            }
+        }
+        catch (Throwable throwable)
+        {
+            close();
+            if (isNotFoundError(throwable))
+            {
+                return;
+            }
+            logger.warn("Exception reading CommitLog", throwable);
+            throw new RuntimeException(throwable);
+        }
+    }
+
+    public static RandomAccessReader reader(org.apache.cassandra.spark.cdc.CommitLog log)
+    {
+        return new CdcRandomAccessReader(log);
+    }
+
+    private void readHeader() throws IOException
+    {
+        long startTimeNanos = System.nanoTime();
+        try
+        {
+            descriptor = CommitLogDescriptor.readHeader(reader, DatabaseDescriptor.getEncryptionContext());
+        }
+        catch (IOException exception)
+        {
+            // Let recover deal with it
+            logger.warn("IOException reading CommitLog header", exception);
+        }
+        if (descriptor == null)
+        {
+            // Don't care about whether or not the handler thinks we can continue. We can't without descriptor.
+            // Whether or not we can continue depends on whether this is the last segment.
+            handleUnrecoverableError(new CommitLogReadException(
+                    String.format("Could not read commit log descriptor in file %s", log.name()),
+                    CommitLogReadErrorReason.UNRECOVERABLE_DESCRIPTOR_ERROR,
+                    false));
+        }
+        else
+        {
+            logger.info("Read log header", "segmentId", descriptor.id,
+                                         "compression", descriptor.compression,
+                                             "version", descriptor.version,
+                                    "messagingVersion", descriptor.getMessagingVersion(),
+                                           "timeNanos", System.nanoTime() - startTimeNanos);
+        }
+    }
+
+    private void read()
+    {
+        try
+        {
+            readCommitLogSegment();
+        }
+        catch (Throwable throwable)
+        {
+            if (isNotFoundError(throwable))
+            {
+                return;
+            }
+            Throwable cause = ThrowableUtils.rootCause(throwable);
+            logger.warn("Exception reading CommitLog", cause);
+            throw new RuntimeException(cause);
+        }
+    }
+
+    private boolean isNotFoundError(Throwable throwable)
+    {
+        TransportFailureException transportEx = ThrowableUtils.rootCause(throwable, TransportFailureException.class);
+        if (transportEx != null && transportEx.isNotFound())
+        {
+            // Underlying CommitLog may have been removed before/during reading
+            // this should only happen when CommitLog is old and can be removed
+            logger.warn("CommitLog not found, assuming removed by underlying storage", transportEx);
+            return true;
+        }
+        return false;
+    }
+
+    /**
+     * Reads mutations from file, handing them off to handler
+     *
+     * @throws IOException IOException
+     */
+    private void readCommitLogSegment() throws IOException
+    {
+        long startTimeNanos = System.nanoTime();
+        SeekableCommitLogSegmentReader segmentReader;
+        try
+        {
+            segmentReader = new SeekableCommitLogSegmentReader(this, descriptor, reader, logger, false);
+        }
+        catch (Exception exception)
+        {
+            handleUnrecoverableError(new CommitLogReadException(
+                    String.format("Unable to create segment reader for commit log file: %s", exception),
+                    CommitLogReadErrorReason.UNRECOVERABLE_UNKNOWN_ERROR,
+                    false));
+            return;
+        }
+
+        try
+        {
+            if (descriptor.id == highWaterMark.segmentId() && reader.getFilePointer() < highWaterMark.position())
+            {
+                segmentReader.seek(highWaterMark.position());
+            }
+
+            for (CommitLogSegmentReader.SyncSegment syncSegment : segmentReader)
+            {
+                // Only tolerate truncationSerializationHeader if we allow in both global and segment
+                // statusTracker.tolerateErrorsInSection = tolerateTruncation && syncSegment.toleratesErrorsInSection;
+
+                statusTracker.errorContext = String.format("Next section at %d in %s",
+                                                           syncSegment.fileStartPosition, descriptor.fileName());
+
+                readSection(syncSegment.input, syncSegment.endPosition, descriptor);
+
+                // Track the position at end of previous section after successfully reading mutations,
+                // so we can update highwater mark after reading
+                position = (int) reader.getFilePointer();
+
+                if (!statusTracker.shouldContinue())
+                {
+                    break;
+                }
+            }
+        }
+        // Unfortunately CommitLogSegmentReader.SegmentIterator (for-loop) cannot throw a checked exception,
+        // so we check to see if a RuntimeException is wrapping an IOException
+        catch (RuntimeException exception)
+        {
+            if (exception.getCause() instanceof IOException)
+            {
+                throw (IOException) exception.getCause();
+            }
+            throw exception;
+        }
+        logger.info("Finished reading commit log", "updates", updates.size(),
+                                                 "timeNanos", System.nanoTime() - startTimeNanos);
+    }
+
+    public boolean skip() throws IOException
+    {
+        if (shouldSkip(reader))
+        {
+            logger.info("Skipping playback of empty log");
+            return true;
+        }
+
+        // Just transform from the file name (no reading of headSkipping playback of empty log:ers) to determine version
+        long segmentIdFromFilename = CommitLogDescriptor.fromFileName(log.name()).id;
+
+        if (segmentIdFromFilename != descriptor.id)
+        {
+            CommitLogReadException readException = new CommitLogReadException(
+                    String.format("Segment id mismatch (filename %d, descriptor %d) in file %s",
+                                  segmentIdFromFilename, descriptor.id, log.name()),
+                    CommitLogReadErrorReason.RECOVERABLE_DESCRIPTOR_ERROR,
+                    false);
+            return shouldSkipSegmentOnError(readException);
+        }
+
+        return shouldSkipSegmentId();
+    }
+
+    /**
+     * Peek the next 8 bytes to determine if it reaches the end of the file.
+     * It should <b>only</b> be called immediately after reading the CommitLog header.
+     *
+     * @return true to skip; otherwise, return false
+     * @throws IOException io exception
+     */
+    private static boolean shouldSkip(RandomAccessReader reader) throws IOException
+    {
+        try
+        {
+            reader.mark();  // Mark position
+            int end = reader.readInt();
+            long filecrc = reader.readInt() & 0xFFFFFFFFL;
+            return end == 0 && filecrc == 0;
+        }
+        catch (EOFException exception)
+        {
+            // No data to read
+            return true;
+        }
+        finally
+        {
+            // Return to marked position before reading mutations
+            reader.reset();
+        }
+    }
+
+    /**
+     * Any segment with id >= minPosition.segmentId is a candidate for read
+     */
+    private boolean shouldSkipSegmentId()
+    {
+        logger.debug("Reading commit log", "version", descriptor.version,
+                                  "messagingVersion", descriptor.getMessagingVersion(),
+                                       "compression", descriptor.compression);
+
+        if (highWaterMark.segmentId() > descriptor.id)
+        {
+            logger.info("Skipping read of fully-flushed log", "segmentId", descriptor.id,
+                                                           "minSegmentId", highWaterMark.segmentId());
+            return true;
+        }
+        else
+        {
+            return false;
+        }
+    }
+
+    /**
+     * Reads a section of a file containing mutations
+     *
+     * @param reader FileDataInput / logical buffer containing CommitLog mutations
+     * @param end    logical numeric end of the segment being read
+     * @param desc   Descriptor for CommitLog serialization
+     */
+    private void readSection(FileDataInput reader, int end, CommitLogDescriptor desc) throws IOException
+    {
+        while (statusTracker.shouldContinue() && reader.getFilePointer() < end && !reader.isEOF())
+        {
+            int mutationStart = (int) reader.getFilePointer();
+            logger.trace("Reading mutation at", "position", mutationStart);
+
+            long claimedCRC32;
+            int serializedSize;
+            try
+            {
+                // We rely on reading serialized size == 0 (LEGACY_END_OF_SEGMENT_MARKER) to identify the end
+                // of a segment, which happens naturally due to the 0 padding of the empty segment on creation.
+                // However, it's possible with 2.1 era CommitLogs that the last mutation ended less than 4 bytes
+                // from the end of the file, which means that we'll be unable to read a full int and instead
+                // read an EOF here.
+                if (end - reader.getFilePointer() < 4)
+                {
+                    logger.trace("Not enough bytes left for another mutation in this CommitLog section, continuing");
+                    statusTracker.requestTermination();
+                    return;
+                }
+
+                // Any of the reads may hit EOF
+                serializedSize = reader.readInt();
+                if (serializedSize == LEGACY_END_OF_SEGMENT_MARKER)
+                {
+                    logger.trace("Encountered end of segment marker at", "position", reader.getFilePointer());
+                    statusTracker.requestTermination();
+                    return;
+                }
+
+                // Mutation must be at LEAST 10 bytes:
+                //   3 for a non-empty Keyspace
+                //   3 for a Key (including the 2-byte length from writeUTF/writeWithShortLength)
+                //   4 bytes for column count
+                // This prevents CRC by being fooled by special-case garbage in the file; see CASSANDRA-2128
+                if (serializedSize < 10)
+                {
+                    if (shouldSkipSegmentOnError(new CommitLogReadException(
+                            String.format("Invalid mutation size %d at %d in %s",
+                                          serializedSize, mutationStart, statusTracker.errorContext),
+                            CommitLogReadErrorReason.MUTATION_ERROR,
+                            statusTracker.tolerateErrorsInSection)))
+                    {
+                        statusTracker.requestTermination();
+                    }
+                    return;
+                }
+
+                long claimedSizeChecksum = CommitLogFormat.calculateClaimedChecksum(reader);
+                checksum.reset();
+                CommitLogFormat.updateChecksum(checksum, serializedSize);
+
+                if (checksum.getValue() != claimedSizeChecksum)
+                {
+                    if (shouldSkipSegmentOnError(new CommitLogReadException(
+                            String.format("Mutation size checksum failure at %d in %s",
+                                          mutationStart, statusTracker.errorContext),
+                            CommitLogReadErrorReason.MUTATION_ERROR,
+                            statusTracker.tolerateErrorsInSection)))
+                    {
+                        statusTracker.requestTermination();
+                    }
+                    return;
+                }
+
+                if (serializedSize > buffer.length)
+                {
+                    buffer = new byte[(int) (1.2 * serializedSize)];
+                }
+                reader.readFully(buffer, 0, serializedSize);
+
+                claimedCRC32 = CommitLogFormat.calculateClaimedCRC32(reader);
+            }
+            catch (EOFException exception)
+            {
+                if (shouldSkipSegmentOnError(new CommitLogReadException(
+                        String.format("Unexpected end of segment at %d in %s",
+                                      mutationStart, statusTracker.errorContext),
+                        CommitLogReadErrorReason.EOF,
+                        statusTracker.tolerateErrorsInSection)))
+                {
+                    statusTracker.requestTermination();
+                }
+                return;
+            }
+
+            checksum.update(buffer, 0, serializedSize);
+            if (claimedCRC32 != checksum.getValue())
+            {
+                if (shouldSkipSegmentOnError(new CommitLogReadException(
+                        String.format("Mutation checksum failure at %d in %s",
+                                      mutationStart, statusTracker.errorContext),
+                        CommitLogReadErrorReason.MUTATION_ERROR,
+                        statusTracker.tolerateErrorsInSection)))
+                {
+                    statusTracker.requestTermination();
+                }
+                continue;
+            }
+
+            int mutationPosition = (int) reader.getFilePointer();
+            readMutationInternal(buffer, serializedSize, mutationPosition, desc);
+            statusTracker.addProcessedMutation();
+        }
+    }
+
+    /**
+     * Deserializes and passes a Mutation to the ICommitLogReadHandler requested
+     *
+     * @param inputBuffer      raw byte array w/Mutation data
+     * @param size             deserialized size of mutation
+     * @param mutationPosition filePointer offset of end of mutation within CommitLogSegment
+     * @param descriptor             CommitLogDescriptor being worked on
+     */
+    @VisibleForTesting
+    private void readMutationInternal(byte[] inputBuffer,
+                                      int size,
+                                      int mutationPosition,
+                                      CommitLogDescriptor descriptor) throws IOException
+    {
+        // For now, we need to go through the motions of deserializing the mutation to determine its size and move
+        // the file pointer forward accordingly, even if we're behind the requested minPosition within this SyncSegment
+
+        Mutation mutation;
+        try (RebufferingInputStream bufferIn = new DataInputBuffer(inputBuffer, 0, size))
+        {
+            mutation = Mutation.serializer.deserialize(bufferIn,
+                                                       descriptor.getMessagingVersion(),
+                                                       DeserializationHelper.Flag.LOCAL);
+            // Double-check that what we read is still valid for the current schema
+            for (PartitionUpdate update : mutation.getPartitionUpdates())
+            {
+                update.validate();
+            }
+        }
+        catch (UnknownTableException exception)
+        {
+            if (exception.id == null)
+            {
+                return;
+            }
+            // We see many unknown table exception logs when we skip over mutations from other tables
+            logger.trace("Invalid mutation", exception);
+            return;
+        }
+        catch (Throwable throwable)
+        {
+            JVMStabilityInspector.inspectThrowable(throwable);
+            Path path = Files.createTempFile("mutation", "dat");
+
+            try (DataOutputStream out = new DataOutputStream(Files.newOutputStream(path)))
+            {
+                out.write(inputBuffer, 0, size);
+            }
+
+            // Checksum passed so this error can't be permissible
+            handleUnrecoverableError(new CommitLogReadException(
+                    String.format("Unexpected error deserializing mutation; saved to %s. "
+                                + "This may be caused by replaying a mutation against a table with the same name but incompatible schema. "
+                                + "Exception follows: %s", path, throwable),
+                    CommitLogReadErrorReason.MUTATION_ERROR,
+                    false));
+            return;
+        }
+
+        logger.trace("Read mutation for", () -> "keyspace", mutation::getKeyspaceName,
+                                               () -> "key", mutation::key,
+                                          () -> "mutation", () -> mutation.getPartitionUpdates().stream()
+                                                                                                .map(AbstractBTreePartition::toString)
+                                                                                                .collect(Collectors.joining(", ", "{", "}")));
+        handleMutation(mutation, size, mutationPosition, descriptor);
+    }
+
+    public boolean isReadable()
+    {
+        return updates != null;
+    }
+
+    public void close()
+    {
+        if (updates == null)
+        {
+            return;
+        }
+
+        try
+        {
+            reader.close();
+            reader = null;
+            updates = null;
+        }
+        catch (Throwable throwable)
+        {
+            logger.error("Unexpected exception closing reader", throwable);
+        }
+    }
+
+    public int compareTo(@NotNull BufferingCommitLogReader that)
+    {
+        return Long.compare(this.descriptor.id, that.descriptor.id);
+    }
+
+    /**
+     * Helper methods to deal with changing formats of internals of the CommitLog without polluting deserialization code
+     */
+    private static class CommitLogFormat
+    {
+        public static long calculateClaimedChecksum(FileDataInput input) throws IOException
+        {
+            return input.readInt() & 0xFFFFFFFFL;
+        }
+
+        public static void updateChecksum(CRC32 checksum, int serializedSize)
+        {
+            updateChecksumInt(checksum, serializedSize);
+        }
+
+        public static long calculateClaimedCRC32(FileDataInput input) throws IOException
+        {
+            return input.readInt() & 0xFFFFFFFFL;
+        }
+    }
+
+    private static class ReadStatusTracker
+    {
+        private int mutationsLeft;
+        public String errorContext = "";         // CHECKSTYLE IGNORE: Public mutable field
+        public boolean tolerateErrorsInSection;  // CHECKSTYLE IGNORE: Public mutable field
+        private boolean error;
+
+        ReadStatusTracker(int mutationLimit, boolean tolerateErrorsInSection)
+        {
+            this.error = false;
+            this.mutationsLeft = mutationLimit;
+            this.tolerateErrorsInSection = tolerateErrorsInSection;
+        }
+
+        public void addProcessedMutation()
+        {
+            if (mutationsLeft == ALL_MUTATIONS)
+            {
+                return;
+            }
+            --mutationsLeft;
+        }
+
+        public boolean shouldContinue()
+        {
+            return !error && mutationsLeft != 0;
+        }
+
+        public void requestTermination()
+        {
+            error = true;
+        }
+    }
+
+    /**
+     * @return result object wrapping list of updates buffered and the final highwater marker position
+     */
+    public Result result()
+    {
+        return new Result(this);
+    }
+
+    public static final class Result
+    {
+        private final List<PartitionUpdateWrapper> updates;
+        private final org.apache.cassandra.spark.cdc.CommitLog.Marker marker;
+
+        private Result(BufferingCommitLogReader reader)
+        {
+            updates = reader.updates;
+            marker = reader.log.markerAt(reader.descriptor.id, reader.position);
+        }
+
+        public List<PartitionUpdateWrapper> updates()
+        {
+            return updates;
+        }
+
+        public org.apache.cassandra.spark.cdc.CommitLog.Marker marker()
+        {
+            return marker;
+        }
+    }
+
+    // CommitLog Read Handler
+
+    public boolean shouldSkipSegmentOnError(CommitLogReadException exception)
+    {
+        logger.warn("CommitLog error on shouldSkipSegment", exception);
+        return false;
+    }
+
+    public void handleUnrecoverableError(CommitLogReadException exception) throws IOException
+    {
+        logger.error("CommitLog unrecoverable error", exception);
+        statusTracker.requestTermination();
+        throw exception;
+    }
+
+    public void handleMutation(Mutation mutation, int size, int mutationPosition, CommitLogDescriptor descriptor)
+    {
+        mutation.getPartitionUpdates().stream()
+                .filter(this::filter)
+                .map(update -> Pair.create(update, maxTimestamp(update)))
+                .filter(this::withinTimeWindow)
+                .map(this::wrapUpdate)
+                .forEach(updates::add);
+    }
+
+    private long maxTimestamp(PartitionUpdate update)
+    {
+        // Row deletion
+        if (update.rowCount() == 1 && !update.lastRow().deletion().isLive())
+        {
+            return update.lastRow().deletion().time().markedForDeleteAt();
+        }
+        else
+        {
+            return update.maxTimestamp();
+        }
+    }
+
+    private PartitionUpdateWrapper wrapUpdate(Pair<PartitionUpdate, Long> update)
+    {
+        return wrapUpdate(update.left, update.right);
+    }
+
+    private PartitionUpdateWrapper wrapUpdate(PartitionUpdate update, long maxTimestampMicros)
+    {
+        return new PartitionUpdateWrapper(table, update, maxTimestampMicros);
+    }
+
+    /**
+     * @param update the partition update
+     * @return true if this is a mutation we are looking for
+     */
+    private boolean filter(PartitionUpdate update)
+    {
+        return isTable(update) && withinRange(update);
+    }
+
+    private boolean isTable(PartitionUpdate update)
+    {
+        return update.metadata().keyspace.equals(table.keyspace)
+            && update.metadata().name.equals(table.name);
+    }
+
+    private boolean withinTimeWindow(Pair<PartitionUpdate, Long> update)
+    {
+        boolean shouldInclude = withinTimeWindow(update.right);
+        if (!shouldInclude)
+        {
+            logger.info("Exclude the update due to out of the allowed time window.", "update", update.left);
+        }
+        return shouldInclude;
+    }
+
+    private boolean withinTimeWindow(long maxTimestampMicros)
+    {
+        if (offsetFilter == null)
+        {
+            return true;
+        }
+        return offsetFilter.overlaps(maxTimestampMicros);
+    }
+
+    /**
+     * @param update a CommitLog PartitionUpdate
+     * @return true if PartitionUpdate overlaps with the Spark worker token range
+     */
+    private boolean withinRange(PartitionUpdate update)
+    {
+        if (sparkRangeFilter == null)
+        {
+            return true;
+        }
+
+        BigInteger token = ReaderUtils.tokenToBigInteger(update.partitionKey().getToken());
+        return !sparkRangeFilter.skipPartition(token);
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/db/commitlog/PartitionUpdateWrapper.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/db/commitlog/PartitionUpdateWrapper.java
new file mode 100644
index 0000000..067452b
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/db/commitlog/PartitionUpdateWrapper.java
@@ -0,0 +1,219 @@
+/*
+ * 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.cassandra.db.commitlog;
+
+import java.nio.ByteBuffer;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.Digest;
+import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.spark.cdc.IPartitionUpdateWrapper;
+import org.apache.cassandra.spark.reader.CdcScannerBuilder;
+import org.apache.cassandra.spark.reader.Scannable;
+import org.jetbrains.annotations.NotNull;
+
+public class PartitionUpdateWrapper implements IPartitionUpdateWrapper, Scannable, Comparable<PartitionUpdateWrapper>
+{
+    public final String keyspace;
+    public final String table;
+    private final TableMetadata tableMetadata;
+    private final PartitionUpdate update;
+    private final ByteBuffer digestBytes;
+    private final long maxTimestampMicros;
+    private final int dataSize;
+
+    public PartitionUpdateWrapper(TableMetadata tableMetadata, PartitionUpdate update, long maxTimestampMicros)
+    {
+        this.tableMetadata = tableMetadata;
+        this.update = update;
+        this.keyspace = update.metadata().keyspace;
+        this.table = update.metadata().name;
+        this.maxTimestampMicros = maxTimestampMicros;
+        Digest digest = Digest.forReadResponse();
+        UnfilteredRowIterators.digest(update.unfilteredIterator(), digest, MessagingService.current_version);
+        this.digestBytes = ByteBuffer.wrap(digest.digest());
+        this.dataSize = 18 /* = 8 + 4 + 2 + 4 */ + digestBytes.remaining() + update.dataSize();
+    }
+
+    // For deserialization
+    private PartitionUpdateWrapper(TableMetadata tableMetadata,
+                                   PartitionUpdate update,
+                                   String keyspace,
+                                   String table,
+                                   long maxTimestampMicros,
+                                   ByteBuffer digestBytes,
+                                   int dataSize)
+    {
+        this.tableMetadata = tableMetadata;
+        this.update = update;
+        this.keyspace = keyspace;
+        this.table = table;
+        this.maxTimestampMicros = maxTimestampMicros;
+        this.digestBytes = digestBytes;
+        this.dataSize = dataSize;
+    }
+
+    public DecoratedKey partitionKey()
+    {
+        return update.partitionKey();
+    }
+
+    @Override
+    public long maxTimestampMicros()
+    {
+        return maxTimestampMicros;
+    }
+
+    @Override
+    public int dataSize()
+    {
+        return dataSize;
+    }
+
+    public PartitionUpdate partitionUpdate()
+    {
+        return update;
+    }
+
+    public ByteBuffer digest()
+    {
+        return digestBytes;
+    }
+
+    @Override
+    public ISSTableScanner scanner()
+    {
+        return new CdcScannerBuilder.CdcScanner(tableMetadata, update);
+    }
+
+    // TODO: Add proper equals and hashCode impl for PartitionUpdate in OSS
+    @Override
+    public int hashCode()
+    {
+        return digestBytes.hashCode();
+    }
+
+    @Override
+    public boolean equals(Object other)
+    {
+        if (this == other)
+        {
+            return true;
+        }
+        else if (other instanceof PartitionUpdateWrapper)
+        {
+            return this.digestBytes.equals(((PartitionUpdateWrapper) other).digestBytes);
+        }
+        else
+        {
+            return false;
+        }
+    }
+
+    public int compareTo(@NotNull PartitionUpdateWrapper that)
+    {
+        return Long.compare(this.maxTimestampMicros, that.maxTimestampMicros);
+    }
+
+    public static class Serializer extends com.esotericsoftware.kryo.Serializer<PartitionUpdateWrapper>
+    {
+        final TableMetadata metadata;
+        final boolean includePartitionUpdate;
+
+        public Serializer(String keyspace, String table)
+        {
+            this(keyspace, table, false);
+        }
+
+        public Serializer(String keyspace, String table, boolean includePartitionUpdate)
+        {
+            this.metadata = Schema.instance.getTableMetadata(keyspace, table);
+            this.includePartitionUpdate = includePartitionUpdate;
+        }
+
+        public Serializer(TableMetadata metadata)
+        {
+            this(metadata, false);
+        }
+
+        public Serializer(TableMetadata metadata, boolean includePartitionUpdate)
+        {
+            this.metadata = metadata;
+            this.includePartitionUpdate = includePartitionUpdate;
+        }
+
+        @Override
+        public PartitionUpdateWrapper read(Kryo kryo, Input in, Class type)
+        {
+            long maxTimestampMicros = in.readLong();
+            int size = in.readInt();
+
+            // Read digest
+            ByteBuffer digest = ByteBuffer.wrap(in.readBytes(in.readShort()));
+
+            PartitionUpdate partitionUpdate = null;
+            if (includePartitionUpdate)
+            {
+                // Read partition update
+                partitionUpdate = PartitionUpdate.fromBytes(ByteBuffer.wrap(in.readBytes(in.readInt())), MessagingService.current_version);
+            }
+
+            String keyspace = in.readString();
+            String table = in.readString();
+
+            return new PartitionUpdateWrapper(metadata, partitionUpdate, keyspace, table, maxTimestampMicros, digest, size);
+        }
+
+        @Override
+        public void write(Kryo kryo, Output out, PartitionUpdateWrapper update)
+        {
+            out.writeLong(update.maxTimestampMicros);  // 8 bytes
+            out.writeInt(update.dataSize());           // 4 bytes
+
+            // Write digest
+            byte[] bytes = new byte[update.digestBytes.remaining()];
+            update.digestBytes.get(bytes);
+            out.writeShort(bytes.length);              // 2 bytes
+            out.writeBytes(bytes);                     // Variable bytes
+            update.digestBytes.clear();
+
+            // Write partition update
+            if (includePartitionUpdate)
+            {
+                ByteBuffer buffer = PartitionUpdate.toBytes(update.update, MessagingService.current_version);
+                byte[] array = new byte[buffer.remaining()];
+                buffer.get(array);
+                out.writeInt(array.length);            // 4 bytes
+                out.writeBytes(array);                 // Variable bytes
+            }
+
+            out.writeString(update.keyspace);
+            out.writeString(update.table);
+        }
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/db/commitlog/SeekableCommitLogSegmentReader.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/db/commitlog/SeekableCommitLogSegmentReader.java
new file mode 100644
index 0000000..930a8dc
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/db/commitlog/SeekableCommitLogSegmentReader.java
@@ -0,0 +1,185 @@
+/*
+ * 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.cassandra.db.commitlog;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.zip.CRC32;
+
+import com.google.common.collect.AbstractIterator;
+
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.spark.utils.LoggerHelper;
+import org.apache.cassandra.utils.FBUtilities;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Copied and refactored from org.apache.cassandra.db.commitlog.CommitLogSegmentReader to deserialize CommitLog segments.
+ * 'seek' method added so we can efficiently seek to previous location without reading and deserializing previous segments.
+ */
+public class SeekableCommitLogSegmentReader implements Iterable<CommitLogSegmentReader.SyncSegment>
+{
+    private final CommitLogReadHandler handler;
+    private final CommitLogDescriptor descriptor;
+    private final RandomAccessReader reader;
+    private final CommitLogSegmentReader.Segmenter segmenter;
+    private final LoggerHelper logger;
+    private final boolean tolerateTruncation;
+
+    /**
+     * Ending position of the current sync section
+     */
+    protected int end;
+
+    protected SeekableCommitLogSegmentReader(CommitLogReadHandler handler,
+                                             CommitLogDescriptor descriptor,
+                                             RandomAccessReader reader,
+                                             LoggerHelper logger,
+                                             boolean tolerateTruncation)
+    {
+        this.handler = handler;
+        this.descriptor = descriptor;
+        this.reader = reader;
+        this.logger = logger;
+        this.tolerateTruncation = tolerateTruncation;
+
+        end = (int) reader.getFilePointer();
+        if (descriptor.getEncryptionContext().isEnabled())
+        {
+            throw new UnsupportedOperationException("Encrypted CommitLogs currently not supported");
+        }
+        else if (descriptor.compression != null)
+        {
+            segmenter = new CommitLogSegmentReader.CompressedSegmenter(descriptor, reader);
+        }
+        else
+        {
+            segmenter = new CommitLogSegmentReader.NoOpSegmenter(reader);
+        }
+    }
+
+    public void seek(int newPosition)
+    {
+        // The SegmentIterator will seek to the new position in the reader when readSyncMarker is next called
+        end = newPosition;
+    }
+
+    protected class SegmentIterator extends AbstractIterator<CommitLogSegmentReader.SyncSegment>
+    {
+        protected CommitLogSegmentReader.SyncSegment computeNext()
+        {
+            while (true)
+            {
+                try
+                {
+                    int currentStart = end;
+                    end = readSyncMarker(descriptor, currentStart, reader);
+                    if (end < 0)
+                    {
+                        return endOfData();
+                    }
+                    if (end > reader.length())
+                    {
+                        // The CRC was good (meaning it was good when it was written and still looks legit),
+                        // but the file is truncated now. Try to grab and use as much of the file as possible,
+                        // which might be nothing if the end of the file truly is corrupt.
+                        end = (int) reader.length();
+                    }
+                    return segmenter.nextSegment(currentStart + CommitLogSegment.SYNC_MARKER_SIZE, end);
+                }
+                catch (CommitLogSegmentReader.SegmentReadException exception)
+                {
+                    try
+                    {
+                        handler.handleUnrecoverableError(new CommitLogReadHandler.CommitLogReadException(
+                                exception.getMessage(),
+                                CommitLogReadHandler.CommitLogReadErrorReason.UNRECOVERABLE_DESCRIPTOR_ERROR,
+                                !exception.invalidCrc && tolerateTruncation));
+                    }
+                    catch (IOException ioException)
+                    {
+                        throw new RuntimeException(ioException);
+                    }
+                }
+                catch (IOException exception)
+                {
+                    try
+                    {
+                        boolean tolerateErrorsInSection = tolerateTruncation & segmenter.tolerateSegmentErrors(end, reader.length());
+                        // If no exception is thrown, the while loop will continue
+                        handler.handleUnrecoverableError(new CommitLogReadHandler.CommitLogReadException(
+                                exception.getMessage(),
+                                CommitLogReadHandler.CommitLogReadErrorReason.UNRECOVERABLE_DESCRIPTOR_ERROR,
+                                tolerateErrorsInSection));
+                    }
+                    catch (IOException ioException)
+                    {
+                        throw new RuntimeException(ioException);
+                    }
+                }
+            }
+        }
+    }
+
+    private int readSyncMarker(CommitLogDescriptor descriptor, int offset, RandomAccessReader reader) throws IOException
+    {
+        if (offset > reader.length() - CommitLogSegment.SYNC_MARKER_SIZE)
+        {
+            // There was no room in the segment to write a final header. No data could be present here.
+            return -1;
+        }
+        long current = reader.getFilePointer();
+        if (offset != current)
+        {
+            long timeNanos = System.nanoTime();
+            reader.seek(offset);
+            logger.info("Seek to position", "from", current, "to", offset, "timeNanos", System.nanoTime() - timeNanos);
+        }
+        CRC32 crc = new CRC32();
+        FBUtilities.updateChecksumInt(crc, (int) (descriptor.id & 0xFFFFFFFFL));
+        FBUtilities.updateChecksumInt(crc, (int) (descriptor.id >>> 32));
+        FBUtilities.updateChecksumInt(crc, (int) reader.getPosition());
+        int end = reader.readInt();
+        long fileCRC = reader.readInt() & 0xFFFFFFFFL;
+        if (crc.getValue() != fileCRC)
+        {
+            if (end != 0 || fileCRC != 0)
+            {
+                String message = String.format("Encountered bad header at position %d of commit log %s, with invalid CRC. "
+                                             + "The end of segment marker should be zero.", offset, reader.getPath());
+                throw new CommitLogSegmentReader.SegmentReadException(message, true);
+            }
+            return -1;
+        }
+        else if (end < offset || reader.length() < end)
+        {
+            String message = String.format("Encountered bad header at position %d of commit log %s, with bad position but valid CRC",
+                                           offset, reader.getPath());
+            throw new CommitLogSegmentReader.SegmentReadException(message, false);
+        }
+        return end;
+    }
+
+    @NotNull
+    public Iterator<CommitLogSegmentReader.SyncSegment> iterator()
+    {
+        return new SegmentIterator();
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/io/sstable/SSTableTombstoneWriter.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/io/sstable/SSTableTombstoneWriter.java
new file mode 100644
index 0000000..aa39324
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/io/sstable/SSTableTombstoneWriter.java
@@ -0,0 +1,512 @@
+/*
+ * 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.cassandra.io.sstable;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.SortedSet;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import org.apache.cassandra.bridge.CassandraSchema;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.cql3.UpdateParameters;
+import org.apache.cassandra.cql3.functions.UDHelper;
+import org.apache.cassandra.cql3.functions.types.TypeCodec;
+import org.apache.cassandra.cql3.statements.Bound;
+import org.apache.cassandra.cql3.statements.DeleteStatement;
+import org.apache.cassandra.cql3.statements.ModificationStatement;
+import org.apache.cassandra.cql3.statements.schema.CreateTableStatement;
+import org.apache.cassandra.cql3.statements.schema.CreateTypeStatement;
+import org.apache.cassandra.db.Clustering;
+import org.apache.cassandra.db.ClusteringBound;
+import org.apache.cassandra.db.ClusteringComparator;
+import org.apache.cassandra.db.Slice;
+import org.apache.cassandra.db.Slices;
+import org.apache.cassandra.db.SystemKeyspace;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.schema.Functions;
+import org.apache.cassandra.schema.KeyspaceMetadata;
+import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.SchemaConstants;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.schema.Tables;
+import org.apache.cassandra.schema.Types;
+import org.apache.cassandra.schema.Views;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+/**
+ * Re-write of CQLSSTableWriter for writing tombstones to an SSTable for testing
+ */
+public final class SSTableTombstoneWriter implements Closeable
+{
+    private static final ByteBuffer UNSET_VALUE = ByteBufferUtil.UNSET_BYTE_BUFFER;
+
+    static
+    {
+        DatabaseDescriptor.clientInitialization(false);
+        // Partitioner is not set in client mode
+        if (DatabaseDescriptor.getPartitioner() == null)
+        {
+            DatabaseDescriptor.setPartitionerUnsafe(Murmur3Partitioner.instance);
+        }
+    }
+
+    private final AbstractSSTableSimpleWriter writer;
+    private final DeleteStatement delete;
+    private final List<ColumnSpecification> boundNames;
+    private final List<TypeCodec> typeCodecs;
+    private final ClusteringComparator comparator;
+
+    private SSTableTombstoneWriter(AbstractSSTableSimpleWriter writer,
+                                   DeleteStatement delete,
+                                   List<ColumnSpecification> boundNames,
+                                   ClusteringComparator comparator)
+    {
+        this.writer = writer;
+        this.delete = delete;
+        this.boundNames = boundNames;
+        this.typeCodecs = boundNames.stream().map(bn -> UDHelper.codecFor(UDHelper.driverType(bn.type)))
+                                    .collect(Collectors.toList());
+        this.comparator = comparator;
+    }
+
+    /**
+     * Returns a new builder for a SSTableTombstoneWriter
+     *
+     * @return the new builder
+     */
+    public static Builder builder()
+    {
+        return new Builder();
+    }
+
+    /**
+     * Adds a new row to the writer.
+     *
+     * This is a shortcut for {@code addRow(Arrays.asList(values))}.
+     *
+     * @param values the row values (corresponding to the bind variables of the
+     *               deletion statement used when creating by this writer)
+     * @throws IOException when adding a row with the given {@code values} fails
+     */
+    public void addRow(Object... values) throws InvalidRequestException, IOException
+    {
+        addRow(Arrays.asList(values));
+    }
+
+    /**
+     * Adds a new row to the writer.
+     * <p>
+     * Each provided value type should correspond to the types of the CQL column the value is for.
+     * The correspondence between java type and CQL type is the same one than the one documented at
+     * www.datastax.com/drivers/java/2.0/apidocs/com/datastax/driver/core/DataType.Name.html#asJavaClass().
+     * <p>
+     * If you prefer providing the values directly as binary, use
+     *
+     * @param values the row values (corresponding to the bind variables of the
+     *               deletion statement used when creating by this writer)
+     */
+    private void addRow(List<Object> values) throws InvalidRequestException, IOException
+    {
+        int size = Math.min(values.size(), boundNames.size());
+        List<ByteBuffer> rawValues = new ArrayList<>(size);
+
+        for (int index = 0; index < size; index++)
+        {
+            Object value = values.get(index);
+            rawValues.add(serialize(value, typeCodecs.get(index)));
+        }
+
+        rawAddRow(rawValues);
+    }
+
+    /**
+     * Adds a new row to the writer given already serialized values.
+     *
+     * This is a shortcut for {@code rawAddRow(Arrays.asList(values))}.
+     *
+     * @param values the row values (corresponding to the bind variables of the
+     *               deletion statement used when creating by this writer) as binary
+     */
+    private void rawAddRow(List<ByteBuffer> values) throws InvalidRequestException, IOException
+    {
+        if (values.size() != boundNames.size())
+        {
+            throw new InvalidRequestException(
+                    String.format("Invalid number of arguments, expecting %d values but got %d",
+                                  boundNames.size(), values.size()));
+        }
+
+        QueryOptions options = QueryOptions.forInternalCalls(null, values);
+        List<ByteBuffer> keys = delete.buildPartitionKeyNames(options);
+
+        long now = System.currentTimeMillis();
+        // NOTE: We ask indexes to not validate values (the last 'false' arg below) because that
+        //       triggers a 'Keyspace.open' and that forces a lot of initialization that we don't want
+        UpdateParameters params = new UpdateParameters(delete.metadata,
+                                                       delete.updatedColumns(),
+                                                       options,
+                                                       delete.getTimestamp(TimeUnit.MILLISECONDS.toMicros(now), options),
+                                                       (int) TimeUnit.MILLISECONDS.toSeconds(now),
+                                                       delete.getTimeToLive(options),
+                                                       Collections.emptyMap());
+
+        if (delete.hasSlices())
+        {
+            // Write out range tombstones
+            SortedSet<ClusteringBound<?>> startBounds = delete.getRestrictions().getClusteringColumnsBounds(Bound.START, options);
+            SortedSet<ClusteringBound<?>> endBounds = delete.getRestrictions().getClusteringColumnsBounds(Bound.END, options);
+            Slices slices = toSlices(startBounds, endBounds);
+
+            try
+            {
+                for (ByteBuffer key : keys)
+                {
+                    for (Slice slice : slices)
+                    {
+                        delete.addUpdateForKey(writer.getUpdateFor(key), slice, params);
+                    }
+                }
+                return;
+            }
+            catch (SSTableSimpleUnsortedWriter.SyncException exception)
+            {
+                // If we use a BufferedWriter and had a problem writing to disk, the IOException has been
+                // wrapped in a SyncException (see BufferedWriter below). We want to extract that IOException.
+                throw (IOException) exception.getCause();
+            }
+        }
+
+        SortedSet<Clustering<?>> clusterings = delete.createClustering(options);
+        try
+        {
+            for (ByteBuffer key : keys)
+            {
+                for (Clustering<?> clustering : clusterings)
+                {
+                    delete.addUpdateForKey(writer.getUpdateFor(key), clustering, params);
+                }
+            }
+        }
+        catch (SSTableSimpleUnsortedWriter.SyncException exception)
+        {
+            // If we use a BufferedWriter and had a problem writing to disk, the IOException has been
+            // wrapped in a SyncException (see BufferedWriter below). We want to extract that IOException.
+            throw (IOException) exception.getCause();
+        }
+    }
+
+    private Slices toSlices(SortedSet<ClusteringBound<?>> startBounds, SortedSet<ClusteringBound<?>> endBounds)
+    {
+        assert startBounds.size() == endBounds.size();
+
+        Slices.Builder builder = new Slices.Builder(comparator);
+
+        Iterator<ClusteringBound<?>> starts = startBounds.iterator();
+        Iterator<ClusteringBound<?>> ends = endBounds.iterator();
+
+        while (starts.hasNext())
+        {
+            Slice slice = Slice.make(starts.next(), ends.next());
+            if (!slice.isEmpty(comparator))
+            {
+                builder.add(slice);
+            }
+        }
+
+        return builder.build();
+    }
+
+    /**
+     * Close this writer.
+     * <p>
+     * This method should be called, otherwise the produced SSTables are not
+     * guaranteed to be complete (and won't be in practice).
+     */
+    public void close() throws IOException
+    {
+        writer.close();
+    }
+
+    @SuppressWarnings("unchecked")
+    private ByteBuffer serialize(Object value, TypeCodec codec)
+    {
+        if (value == null || value == UNSET_VALUE)
+        {
+            return (ByteBuffer) value;
+        }
+
+        return codec.serialize(value, ProtocolVersion.CURRENT);
+    }
+
+    /**
+     * A Builder for a SSTableTombstoneWriter object
+     */
+    public static class Builder
+    {
+        private File directory;
+
+        SSTableFormat.Type formatType = null;
+
+        private CreateTableStatement.Raw schemaStatement;
+        private final List<CreateTypeStatement.Raw> typeStatements;
+        private ModificationStatement.Parsed deleteStatement;
+        private IPartitioner partitioner;
+
+        private boolean sorted = false;
+        private long bufferSizeInMB = 128;
+
+        Builder()
+        {
+            this.typeStatements = new ArrayList<>();
+        }
+
+        /**
+         * The directory where to write the SSTables (mandatory option).
+         *
+         * This is a mandatory option.
+         *
+         * @param directory the directory to use, which should exists and be writable
+         * @return this builder
+         * @throws IllegalArgumentException if {@code directory} doesn't exist or is not writable
+         */
+        public Builder inDirectory(File directory)
+        {
+            if (!directory.exists())
+            {
+                throw new IllegalArgumentException(directory + " doesn't exists");
+            }
+            if (!directory.canWrite())
+            {
+                throw new IllegalArgumentException(directory + " exists but is not writable");
+            }
+
+            this.directory = directory;
+            return this;
+        }
+
+        /**
+         * The schema (CREATE TABLE statement) for the table for which SSTable are to be created.
+         * <p>
+         * Please note that the provided CREATE TABLE statement <b>must</b> use a fully-qualified
+         * table name, one that include the keyspace name.
+         * <p>
+         * This is a mandatory option.
+         *
+         * @param schema the schema of the table for which SSTables are to be created
+         * @return this builder
+         * @throws IllegalArgumentException if {@code schema} is not a valid CREATE TABLE statement
+         *                                  or does not have a fully-qualified table name
+         */
+        public Builder forTable(String schema)
+        {
+            schemaStatement = QueryProcessor.parseStatement(schema, CreateTableStatement.Raw.class, "CREATE TABLE");
+            return this;
+        }
+
+        /**
+         * The partitioner to use.
+         * <p>
+         * By default, {@code Murmur3Partitioner} will be used. If this is not the partitioner used
+         * by the cluster for which the SSTables are created, you need to use this method to
+         * provide the correct partitioner.
+         *
+         * @param partitioner the partitioner to use
+         * @return this builder
+         */
+        public Builder withPartitioner(IPartitioner partitioner)
+        {
+            this.partitioner = partitioner;
+            return this;
+        }
+
+        /**
+         * The DELETE statement defining the values to remove for a given CQL row.
+         * <p>
+         * Please note that the provided DELETE statement <b>must</b> use a fully-qualified
+         * table name, one that include the keyspace name. Moreover, said statement must use
+         * bind variables since these variables will be bound to values by the resulting writer.
+         * <p>
+         * This is a mandatory option.
+         *
+         * @param delete a delete statement that defines the order of column values to use
+         * @return this builder
+         * @throws IllegalArgumentException if {@code deleteStatement} is not a valid deletion statement,
+         *                                  does not have a fully-qualified table name or have no bind variables
+         */
+        public Builder using(String delete)
+        {
+            deleteStatement = QueryProcessor.parseStatement(delete, ModificationStatement.Parsed.class, "DELETE");
+            return this;
+        }
+
+        /**
+         * The size of the buffer to use.
+         * <p>
+         * This defines how much data will be buffered before being written as
+         * a new SSTable. This correspond roughly to the data size that will have the created
+         * SSTable.
+         * <p>
+         * The default is 128MB, which should be reasonable for a 1GB heap. If you experience
+         * OOM while using the writer, you should lower this value.
+         *
+         * @param size the size to use in MB
+         * @return this builder
+         */
+        public Builder withBufferSizeInMB(int size)
+        {
+            bufferSizeInMB = size;
+            return this;
+        }
+
+        public SSTableTombstoneWriter build()
+        {
+            if (directory == null)
+            {
+                throw new IllegalStateException("No ouptut directory specified, you should provide a directory with inDirectory()");
+            }
+            if (schemaStatement == null)
+            {
+                throw new IllegalStateException("Missing schema, you should provide the schema for the SSTable to create with forTable()");
+            }
+            if (deleteStatement == null)
+            {
+                throw new IllegalStateException("No delete statement specified, you should provide a delete statement through using()");
+            }
+
+            TableMetadata tableMetadata = CassandraSchema.apply(schema -> {
+                if (schema.getKeyspaceMetadata(SchemaConstants.SYSTEM_KEYSPACE_NAME) == null)
+                {
+                    schema.load(SystemKeyspace.metadata());
+                }
+
+                String keyspaceName = schemaStatement.keyspace();
+
+                if (schema.getKeyspaceMetadata(keyspaceName) == null)
+                {
+                    schema.load(KeyspaceMetadata.create(keyspaceName,
+                                                        KeyspaceParams.simple(1),
+                                                        Tables.none(),
+                                                        Views.none(),
+                                                        Types.none(),
+                                                        Functions.none()));
+                }
+
+                KeyspaceMetadata ksm = schema.getKeyspaceMetadata(keyspaceName);
+
+                TableMetadata table = ksm.tables.getNullable(schemaStatement.table());
+                if (table == null)
+                {
+                    Types types = createTypes(keyspaceName);
+                    table = createTable(types);
+                    schema.load(ksm.withSwapped(ksm.tables.with(table)).withSwapped(types));
+                }
+                return table;
+            });
+
+            DeleteStatement preparedDelete = prepareDelete();
+            TableMetadataRef ref = TableMetadataRef.forOfflineTools(tableMetadata);
+            AbstractSSTableSimpleWriter writer = sorted
+                    ? new SSTableSimpleWriter(directory, ref, preparedDelete.updatedColumns())
+                    : new SSTableSimpleUnsortedWriter(directory, ref, preparedDelete.updatedColumns(), bufferSizeInMB);
+
+            if (formatType != null)
+            {
+                writer.setSSTableFormatType(formatType);
+            }
+
+            return new SSTableTombstoneWriter(writer, preparedDelete, preparedDelete.getBindVariables(), tableMetadata.comparator);
+        }
+
+        private Types createTypes(String keyspace)
+        {
+            Types.RawBuilder builder = Types.rawBuilder(keyspace);
+            for (CreateTypeStatement.Raw st : typeStatements)
+            {
+                st.addToRawBuilder(builder);
+            }
+            return builder.build();
+        }
+
+        /**
+         * Creates the table according to schema statement
+         *
+         * @param types types this table should be created with
+         */
+        private TableMetadata createTable(Types types)
+        {
+            ClientState state = ClientState.forInternalCalls();
+            CreateTableStatement statement = schemaStatement.prepare(state);
+            statement.validate(ClientState.forInternalCalls());
+
+            TableMetadata.Builder builder = statement.builder(types);
+            if (partitioner != null)
+            {
+                builder.partitioner(partitioner);
+            }
+
+            return builder.build();
+        }
+
+        /**
+         * Prepares delete statement for writing data to SSTable
+         *
+         * @return prepared Delete statement and it's bound names
+         */
+        private DeleteStatement prepareDelete()
+        {
+            ClientState state = ClientState.forInternalCalls();
+            DeleteStatement delete = (DeleteStatement) deleteStatement.prepare(state);
+            delete.validate(state);
+
+            if (delete.hasConditions())
+            {
+                throw new IllegalArgumentException("Conditional statements are not supported");
+            }
+            if (delete.isCounter())
+            {
+                throw new IllegalArgumentException("Counter update statements are not supported");
+            }
+            if (delete.getBindVariables().isEmpty())
+            {
+                throw new IllegalArgumentException("Provided delete statement has no bind variables");
+            }
+
+            return delete;
+        }
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/io/util/CdcRandomAccessReader.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/io/util/CdcRandomAccessReader.java
new file mode 100644
index 0000000..d33535d
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/io/util/CdcRandomAccessReader.java
@@ -0,0 +1,258 @@
+/*
+ * 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.cassandra.io.util;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.spark.cdc.CommitLog;
+import org.apache.cassandra.spark.data.SSTable;
+import org.apache.cassandra.spark.utils.ByteBufferUtils;
+import org.apache.cassandra.spark.utils.IOUtils;
+import org.apache.cassandra.spark.utils.ThrowableUtils;
+import org.apache.cassandra.spark.utils.streaming.SSTableInputStream;
+import org.apache.cassandra.spark.utils.streaming.SSTableSource;
+import org.apache.cassandra.spark.utils.streaming.StreamBuffer;
+import org.apache.cassandra.spark.utils.streaming.StreamConsumer;
+
+public class CdcRandomAccessReader extends RandomAccessReader
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(CdcRandomAccessReader.class);
+
+    private final CommitLog log;
+
+    public CdcRandomAccessReader(CommitLog log)
+    {
+        super(new CdcRebufferer(log));
+        this.log = log;
+    }
+
+    @Override
+    public String getPath()
+    {
+        return log.path();
+    }
+
+    public static class CdcRebufferer implements Rebufferer, Rebufferer.BufferHolder
+    {
+        ByteBuffer buffer;
+        final CommitLog log;
+        final int chunkSize;
+        long offset = 0;
+        final SSTableSource<? extends SSTable> source;
+        private final SSTableInputStream<? extends SSTable> inputStream;
+
+        CdcRebufferer(CommitLog log)
+        {
+            this(log, IOUtils.DEFAULT_CDC_BUFFER_SIZE);
+        }
+
+        CdcRebufferer(CommitLog log, int chunkSize)
+        {
+            Preconditions.checkArgument(chunkSize > 0, "Chunk size must be a positive integer");
+            this.log = log;
+            this.chunkSize = chunkSize;
+            this.buffer = ByteBuffer.allocate(bufferSize());
+            this.source = log.source();
+
+            // We read the CommitLogs sequentially, so we can re-use the SSTableInputStream
+            // to async read ahead and reduce time spent blocking on I/O
+            this.inputStream = new SSTableInputStream<>(source, log.stats());
+        }
+
+        private int bufferSize()
+        {
+            return Math.toIntExact(Math.min(log.maxOffset() - offset, chunkSize));
+        }
+
+        @Override
+        public BufferHolder rebuffer(long position)
+        {
+            offset = position;
+            buffer.clear();
+            int length = bufferSize();
+            if (length < 0)
+            {
+                throw new IllegalStateException(String.format("Read passed maxOffset offset=%d maxOffset=%d",
+                                                              offset, log.maxOffset()));
+            }
+            if (buffer.capacity() != length)
+            {
+                // The buffer size will always be chunkSize or CdcRandomAccessReader.DEFAULT_BUFFER_SIZE until we reach the end
+                buffer = ByteBuffer.allocate(length);
+            }
+
+            long currentPos = inputStream.bytesRead();
+            try
+            {
+                if (offset < currentPos)
+                {
+                    // Attempting to read bytes previously read. In practice, we read the CommitLogs sequentially,
+                    // but we still need to respect random access reader API, it will just require blocking.
+                    int requestLength = buffer.remaining() + 1;
+                    long end = offset + requestLength;
+                    BlockingStreamConsumer streamConsumer = new BlockingStreamConsumer();
+                    source.request(offset, end, streamConsumer);
+                    streamConsumer.getBytes(buffer);
+                    buffer.flip();
+                    return this;
+                }
+
+                if (offset > currentPos)
+                {
+                    // Skip ahead
+                    ByteBufferUtils.skipFully(inputStream, offset - currentPos);
+                }
+
+                inputStream.read(buffer);
+                assert buffer.remaining() == 0;
+                buffer.flip();
+            }
+            catch (IOException exception)
+            {
+                throw new RuntimeException(ThrowableUtils.rootCause(exception));
+            }
+
+            return this;
+        }
+
+        @Override
+        public void closeReader()
+        {
+            offset = -1;
+            close();
+        }
+
+        @Override
+        public void close()
+        {
+            assert offset == -1;  // Reader must be closed at this point
+            inputStream.close();
+            try
+            {
+                log.close();
+            }
+            catch (Exception exception)
+            {
+                LOGGER.error("Exception closing CommitLog", exception);
+            }
+            buffer = null;
+        }
+
+        @Override
+        public ChannelProxy channel()
+        {
+            throw new IllegalStateException("Channel method should not be used");
+        }
+
+        @Override
+        public long fileLength()
+        {
+            return log.maxOffset();
+        }
+
+        @Override
+        public double getCrcCheckChance()
+        {
+            return 0;  // Only valid for compressed files
+        }
+
+        // Buffer holder
+        @Override
+        public ByteBuffer buffer()
+        {
+            return buffer;
+        }
+
+        @Override
+        public long offset()
+        {
+            return offset;
+        }
+
+        @Override
+        public void release()
+        {
+            // Nothing to do, we don't delete buffers before we're closed
+        }
+    }
+
+    public static class BlockingStreamConsumer implements StreamConsumer
+    {
+        private final List<StreamBuffer> buffers;
+        private final CompletableFuture<List<StreamBuffer>> future = new CompletableFuture<>();
+
+        BlockingStreamConsumer()
+        {
+            buffers = new ArrayList<>();
+        }
+
+        /**
+         * This method should be called by the same thread, but synchronized keyword is added to rely on biased locking
+         *
+         * @param buffer StreamBuffer wrapping the bytes
+         */
+        @Override
+        public synchronized void onRead(StreamBuffer buffer)
+        {
+            buffers.add(buffer);
+        }
+
+        @Override
+        public synchronized void onEnd()
+        {
+            future.complete(buffers);
+        }
+
+        @Override
+        public void onError(Throwable throwable)
+        {
+            future.completeExceptionally(throwable);
+        }
+
+        public void getBytes(ByteBuffer destination)
+        {
+            try
+            {
+                for (StreamBuffer buffer : future.get())
+                {
+                    buffer.getBytes(0, destination, buffer.readableBytes());
+                }
+            }
+            catch (InterruptedException exception)
+            {
+                Thread.currentThread().interrupt();
+                throw new RuntimeException(exception);
+            }
+            catch (ExecutionException exception)
+            {
+                throw new RuntimeException(ThrowableUtils.rootCause(exception));
+            }
+        }
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/CqlType.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/CqlType.java
new file mode 100644
index 0000000..0bdf027
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/CqlType.java
@@ -0,0 +1,197 @@
+/*
+ * 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.cassandra.spark.data;
+
+import java.nio.ByteBuffer;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+
+import org.apache.cassandra.bridge.BigNumberConfig;
+import org.apache.cassandra.bridge.CassandraVersion;
+import org.apache.cassandra.cql3.functions.types.CodecRegistry;
+import org.apache.cassandra.cql3.functions.types.DataType;
+import org.apache.cassandra.cql3.functions.types.SettableByIndexData;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.ListType;
+import org.apache.cassandra.db.rows.BufferCell;
+import org.apache.cassandra.db.rows.CellPath;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.serializers.TypeSerializer;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+
+public abstract class CqlType implements CqlField.CqlType
+{
+    public static final CodecRegistry CODEC_REGISTRY = new CodecRegistry();
+
+    @Override
+    public CassandraVersion version()
+    {
+        return CassandraVersion.FOURZERO;
+    }
+
+    public abstract AbstractType<?> dataType();
+
+    public abstract AbstractType<?> dataType(boolean isMultiCell);
+
+    @Override
+    public Object deserialize(ByteBuffer buffer)
+    {
+        return deserialize(buffer, false);
+    }
+
+    @Override
+    public Object deserialize(ByteBuffer buffer, boolean isFrozen)
+    {
+        return toSparkSqlType(serializer().deserialize(buffer));
+    }
+
+    public abstract <T> TypeSerializer<T> serializer();
+
+    @Override
+    public ByteBuffer serialize(Object value)
+    {
+        return serializer().serialize(value);
+    }
+
+    @Override
+    public Object randomValue(int minCollectionSize)
+    {
+        throw CqlField.notImplemented(this);
+    }
+
+    public DataType driverDataType()
+    {
+        return driverDataType(false);
+    }
+
+    public DataType driverDataType(boolean isFrozen)
+    {
+        throw CqlField.notImplemented(this);
+    }
+
+    @Override
+    public org.apache.spark.sql.types.DataType sparkSqlType()
+    {
+        return sparkSqlType(BigNumberConfig.DEFAULT);
+    }
+
+    @Override
+    public org.apache.spark.sql.types.DataType sparkSqlType(BigNumberConfig bigNumberConfig)
+    {
+        throw CqlField.notImplemented(this);
+    }
+
+    @Override
+    public Object sparkSqlRowValue(GenericInternalRow row, int position)
+    {
+        throw CqlField.notImplemented(this);
+    }
+
+    @Override
+    public Object sparkSqlRowValue(Row row, int position)
+    {
+        throw CqlField.notImplemented(this);
+    }
+
+    // Set inner value for UDTs or Tuples
+    public void setInnerValue(SettableByIndexData<?> udtValue, int position, Object value)
+    {
+        throw CqlField.notImplemented(this);
+    }
+
+    @Override
+    public String toString()
+    {
+        return cqlName();
+    }
+
+    @Override
+    public int cardinality(int orElse)
+    {
+        return orElse;
+    }
+
+    @Override
+    public Object toTestRowType(Object value)
+    {
+        return value;
+    }
+
+    @VisibleForTesting
+    public void addCell(org.apache.cassandra.db.rows.Row.Builder rowBuilder,
+                        ColumnMetadata column,
+                        long timestamp,
+                        Object value)
+    {
+        addCell(rowBuilder, column, timestamp, value, null);
+    }
+
+    @VisibleForTesting
+    public void addCell(org.apache.cassandra.db.rows.Row.Builder rowBuilder,
+                        ColumnMetadata column,
+                        long timestamp,
+                        Object value,
+                        CellPath cellPath)
+    {
+        rowBuilder.addCell(BufferCell.live(column, timestamp, serialize(value), cellPath));
+    }
+
+    /**
+     * Tombstone a simple cell, i.e. it does not work on complex types such as non-frozen collection and UDT
+     */
+    @VisibleForTesting
+    public void addTombstone(org.apache.cassandra.db.rows.Row.Builder rowBuilder, ColumnMetadata column, long timestamp)
+    {
+        Preconditions.checkArgument(!column.isComplex(), "The method only works with non-complex columns");
+        addTombstone(rowBuilder, column, timestamp, null);
+    }
+
+    /**
+     * Tombstone an element in multi-cells data types such as non-frozen collection and UDT
+     *
+     * @param cellPath denotes the element to be tombstoned
+     */
+    @VisibleForTesting
+    public void addTombstone(org.apache.cassandra.db.rows.Row.Builder rowBuilder,
+                             ColumnMetadata column,
+                             long timestamp,
+                             CellPath cellPath)
+    {
+        Preconditions.checkArgument(!(column.type instanceof ListType),
+                                    "The method does not support tombstone elements from a List type");
+        rowBuilder.addCell(BufferCell.tombstone(column, timestamp, (int) TimeUnit.MICROSECONDS.toSeconds(timestamp), cellPath));
+    }
+
+    /**
+     * Tombstone the entire complex cell, i.e. non-frozen collection and UDT
+     */
+    @VisibleForTesting
+    public void addComplexTombstone(org.apache.cassandra.db.rows.Row.Builder rowBuilder,
+                                    ColumnMetadata column,
+                                    long deletionTime)
+    {
+        Preconditions.checkArgument(column.isComplex(), "The method only works with complex columns");
+        rowBuilder.addComplexDeletion(column, new DeletionTime(deletionTime, (int) TimeUnit.MICROSECONDS.toSeconds(deletionTime)));
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/NativeType.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/NativeType.java
new file mode 100644
index 0000000..19552a4
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/NativeType.java
@@ -0,0 +1,180 @@
+/*
+ * 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.cassandra.spark.data;
+
+import java.util.Collections;
+import java.util.Set;
+
+import com.esotericsoftware.kryo.io.Output;
+import org.apache.cassandra.bridge.CassandraVersion;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.serializers.TypeSerializer;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+
+public abstract class NativeType extends CqlType implements CqlField.NativeType
+{
+    private final int hashCode;
+
+    protected NativeType()
+    {
+        hashCode = name().hashCode();
+    }
+
+    public CqlField.CqlType.InternalType internalType()
+    {
+        return CqlField.CqlType.InternalType.NativeCql;
+    }
+
+    @Override
+    public boolean isSupported()
+    {
+        return true;
+    }
+
+    @Override
+    public AbstractType<?> dataType()
+    {
+        throw CqlField.notImplemented(this);
+    }
+
+    @Override
+    public Object toSparkSqlType(Object value)
+    {
+        return toSparkSqlType(value, false);
+    }
+
+    @Override
+    public Object toSparkSqlType(Object value, boolean isFrozen)
+    {
+        // All other non-overridden data types work as ordinary Java data types
+        return value;
+    }
+
+    @Override
+    public Object convertForCqlWriter(Object value, CassandraVersion version)
+    {
+        return value;
+    }
+
+    @Override
+    public Object sparkSqlRowValue(GenericInternalRow row, int position)
+    {
+        // We need to convert native types to TestRow types
+        return row.isNullAt(position) ? null : toTestRowType(nativeSparkSqlRowValue(row, position));
+    }
+
+    protected Object nativeSparkSqlRowValue(GenericInternalRow row, int position)
+    {
+        throw CqlField.notImplemented(this);
+    }
+
+    @Override
+    public Object sparkSqlRowValue(Row row, int position)
+    {
+        // We need to convert native types to TestRow types
+        return row.isNullAt(position) ? null : toTestRowType(nativeSparkSqlRowValue(row, position));
+    }
+
+    protected Object nativeSparkSqlRowValue(Row row, int position)
+    {
+        throw CqlField.notImplemented(this);
+    }
+
+    @Override
+    public AbstractType<?> dataType(boolean isMultiCell)
+    {
+        return dataType();
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return hashCode;
+    }
+
+    @Override
+    public boolean equals(Object other)
+    {
+        return other != null && (this == other || this.getClass() == other.getClass());
+    }
+
+    @Override
+    public boolean equals(Object first, Object second)
+    {
+        if (first == second)
+        {
+            return true;
+        }
+        else if (first == null || second == null)
+        {
+            return false;
+        }
+        else
+        {
+            return equalsTo(first, second);
+        }
+    }
+
+    protected boolean equalsTo(Object first, Object second)
+    {
+        return compare(first, second) == 0;
+    }
+
+    @Override
+    public int compare(Object first, Object second)
+    {
+        if (first == null || second == null)
+        {
+            return first == second ? 0 : (first == null ? -1 : 1);
+        }
+        return compareTo(first, second);
+    }
+
+    protected int compareTo(Object first, Object second)
+    {
+        throw CqlField.notImplemented(this);
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public <T> TypeSerializer<T> serializer()
+    {
+        return (TypeSerializer<T>) dataType().getSerializer();
+    }
+
+    @Override
+    public String cqlName()
+    {
+        return name().toLowerCase();
+    }
+
+    @Override
+    public void write(Output output)
+    {
+        CqlField.CqlType.write(this, output);
+        output.writeString(name());
+    }
+
+    public Set<CqlField.CqlUdt> udts()
+    {
+        return Collections.emptySet();
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/complex/CqlCollection.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/complex/CqlCollection.java
new file mode 100644
index 0000000..9e0316f
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/complex/CqlCollection.java
@@ -0,0 +1,248 @@
+/*
+ * 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.cassandra.spark.data.complex;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.commons.lang.builder.EqualsBuilder;
+import org.apache.commons.lang.builder.HashCodeBuilder;
+
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+import org.apache.cassandra.bridge.CassandraBridge;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.spark.data.CqlField;
+import org.apache.cassandra.spark.data.CqlType;
+
+public abstract class CqlCollection extends CqlType implements CqlField.CqlCollection
+{
+    public final List<CqlField.CqlType> types;
+    private final int hashCode;
+
+    CqlCollection(CqlField.CqlType type)
+    {
+        this(Collections.singletonList(type));
+    }
+
+    CqlCollection(CqlField.CqlType... types)
+    {
+        this(Arrays.asList(types));
+    }
+
+    CqlCollection(List<CqlField.CqlType> types)
+    {
+        this.types = new ArrayList<>(types);
+        this.hashCode = new HashCodeBuilder()
+                        .append(types.toArray(new CqlField.CqlType[0]))
+                        .hashCode();
+    }
+
+    @Override
+    public Object toSparkSqlType(Object value)
+    {
+        return toSparkSqlType(value, false);
+    }
+
+    @Override
+    public AbstractType<?> dataType()
+    {
+        return dataType(true);
+    }
+
+    @Override
+    public ByteBuffer serialize(Object value)
+    {
+        return serializer().serialize(value);
+    }
+
+    @Override
+    public Object deserialize(ByteBuffer buffer)
+    {
+        return deserialize(buffer, false);
+    }
+
+    @Override
+    public Object deserialize(ByteBuffer buffer, boolean isFrozen)
+    {
+        return toSparkSqlType(serializer().deserialize(buffer));
+    }
+
+    @Override
+    public boolean isSupported()
+    {
+        return true;
+    }
+
+    public static CqlCollection build(String name, CqlField.CqlType... types)
+    {
+        return build(CqlField.CqlType.InternalType.fromString(name), types);
+    }
+
+    public static CqlCollection build(CqlField.CqlType.InternalType internalType, CqlField.CqlType... types)
+    {
+        if (types.length < 1 || types[0] == null)
+        {
+            throw new IllegalArgumentException("Collection type requires a non-null key data type");
+        }
+
+        switch (internalType)
+        {
+            case Set:
+                return set(types[0]);
+            case List:
+                return list(types[0]);
+            case Map:
+                if (types.length < 2 || types[1] == null)
+                {
+                    throw new IllegalArgumentException("Map collection type requires a non-null value data type");
+                }
+                return map(types[0], types[1]);
+            case Tuple:
+                return tuple(types);
+            default:
+                throw new IllegalArgumentException("Unknown collection type: " + internalType);
+        }
+    }
+
+    public static CqlList list(CqlField.CqlType type)
+    {
+        return new CqlList(type);
+    }
+
+    public static CqlSet set(CqlField.CqlType type)
+    {
+        return new CqlSet(type);
+    }
+
+    public static CqlMap map(CqlField.CqlType keyType, CqlField.CqlType valueType)
+    {
+        return new CqlMap(keyType, valueType);
+    }
+
+    public static CqlTuple tuple(CqlField.CqlType... types)
+    {
+        return new CqlTuple(types);
+    }
+
+    public int size()
+    {
+        return types.size();
+    }
+
+    public List<CqlField.CqlType> types()
+    {
+        return types;
+    }
+
+    public CqlField.CqlType type()
+    {
+        return type(0);
+    }
+
+    public CqlField.CqlType type(int position)
+    {
+        return types.get(position);
+    }
+
+    public CqlFrozen frozen()
+    {
+        return CqlFrozen.build(this);
+    }
+
+    public String cqlName()
+    {
+        return String.format("%s<%s>", internalType().name().toLowerCase(), types.stream()
+                                                                                 .map(CqlField.CqlType::cqlName)
+                                                                                 .collect(Collectors.joining(", ")));
+    }
+
+    @Override
+    public Set<CqlField.CqlUdt> udts()
+    {
+        return types.stream()
+                    .map(CqlField.CqlType::udts)
+                    .flatMap(Collection::stream)
+                    .collect(Collectors.toSet());
+    }
+
+    @Override
+    public String toString()
+    {
+        return cqlName();
+    }
+
+    public static CqlCollection read(CqlField.CqlType.InternalType internalType, Input input, CassandraBridge bridge)
+    {
+        int numTypes = input.readInt();
+        CqlField.CqlType[] types = new CqlField.CqlType[numTypes];
+        for (int type = 0; type < numTypes; type++)
+        {
+            types[type] = CqlField.CqlType.read(input, bridge);
+        }
+        return CqlCollection.build(internalType, types);
+    }
+
+    @Override
+    public void write(Output output)
+    {
+        CqlField.CqlType.write(this, output);
+        output.writeInt(this.types.size());
+        for (CqlField.CqlType type : this.types)
+        {
+            type.write(output);
+        }
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return hashCode;
+    }
+
+    @Override
+    public boolean equals(Object other)
+    {
+        if (other == null)
+        {
+            return false;
+        }
+        if (this == other)
+        {
+            return true;
+        }
+        if (this.getClass() != other.getClass())
+        {
+            return false;
+        }
+
+        CqlCollection that = (CqlCollection) other;
+        return new EqualsBuilder()
+               .append(this.internalType(), that.internalType())
+               .append(this.types, that.types)
+               .isEquals();
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/complex/CqlFrozen.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/complex/CqlFrozen.java
new file mode 100644
index 0000000..9c364c7
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/complex/CqlFrozen.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.cassandra.spark.data.complex;
+
+import java.nio.ByteBuffer;
+import java.util.Set;
+
+import org.apache.commons.lang.builder.EqualsBuilder;
+import org.apache.commons.lang.builder.HashCodeBuilder;
+
+import com.esotericsoftware.kryo.io.Output;
+import org.apache.cassandra.bridge.BigNumberConfig;
+import org.apache.cassandra.bridge.CassandraVersion;
+import org.apache.cassandra.cql3.functions.types.SettableByIndexData;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.serializers.TypeSerializer;
+import org.apache.cassandra.spark.data.CqlField;
+import org.apache.cassandra.spark.data.CqlType;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+import org.apache.spark.sql.types.DataType;
+
+public class CqlFrozen extends CqlType implements CqlField.CqlFrozen
+{
+    private final CqlField.CqlType inner;
+    private final int hashCode;
+
+    public CqlFrozen(CqlField.CqlType inner)
+    {
+        this.inner = inner;
+        this.hashCode = new HashCodeBuilder()
+                .append(internalType().ordinal())
+                .append(inner)
+                .toHashCode();
+    }
+
+    public static CqlFrozen build(CqlField.CqlType inner)
+    {
+        return new CqlFrozen(inner);
+    }
+
+    @Override
+    public boolean isSupported()
+    {
+        return true;
+    }
+
+    @Override
+    public AbstractType<?> dataType()
+    {
+        return ((CqlType) inner()).dataType(false);  // If frozen collection then isMultiCell is false
+    }
+
+    @Override
+    public AbstractType<?> dataType(boolean isMultiCell)
+    {
+        return dataType();
+    }
+
+    @Override
+    public Object toSparkSqlType(Object value)
+    {
+        return inner().toSparkSqlType(value, true);
+    }
+
+    @Override
+    public Object toSparkSqlType(Object value, boolean isFrozen)
+    {
+        return toSparkSqlType(value);
+    }
+
+    @Override
+    public <T> TypeSerializer<T> serializer()
+    {
+        return ((CqlType) inner()).serializer();
+    }
+
+    @Override
+    public Object deserialize(ByteBuffer buffer)
+    {
+        return inner().deserialize(buffer, true);
+    }
+
+    @Override
+    public Object deserialize(ByteBuffer buffer, boolean isFrozen)
+    {
+        return deserialize(buffer);
+    }
+
+    @Override
+    public ByteBuffer serialize(Object value)
+    {
+        return inner().serialize(value);
+    }
+
+    @Override
+    public boolean equals(Object first, Object second)
+    {
+        return inner().equals(first, second);
+    }
+
+    public InternalType internalType()
+    {
+        return InternalType.Frozen;
+    }
+
+    @Override
+    public String name()
+    {
+        return "frozen";
+    }
+
+    public CqlField.CqlType inner()
+    {
+        return inner;
+    }
+
+    public String cqlName()
+    {
+        return String.format("frozen<%s>", inner.cqlName());
+    }
+
+    @Override
+    public DataType sparkSqlType(BigNumberConfig bigNumberConfig)
+    {
+        return inner.sparkSqlType(bigNumberConfig);
+    }
+
+    @Override
+    public Set<CqlField.CqlUdt> udts()
+    {
+        return inner.udts();
+    }
+
+    @Override
+    public Object sparkSqlRowValue(GenericInternalRow row, int position)
+    {
+        return inner.sparkSqlRowValue(row, position);
+    }
+
+    @Override
+    public Object sparkSqlRowValue(Row row, int position)
+    {
+        return inner.sparkSqlRowValue(row, position);
+    }
+
+    @Override
+    public Object toTestRowType(Object value)
+    {
+        return inner.toTestRowType(value);
+    }
+
+    @Override
+    public void setInnerValue(SettableByIndexData<?> udtValue, int position, Object value)
+    {
+        ((CqlType) inner()).setInnerValue(udtValue, position, value);
+    }
+
+    @Override
+    public Object randomValue(int minCollectionSize)
+    {
+        return inner.randomValue(minCollectionSize);
+    }
+
+    @Override
+    public org.apache.cassandra.cql3.functions.types.DataType driverDataType(boolean isFrozen)
+    {
+        return ((CqlType) inner()).driverDataType(true);
+    }
+
+    @Override
+    public Object convertForCqlWriter(Object value, CassandraVersion version)
+    {
+        return inner.convertForCqlWriter(value, version);
+    }
+
+    @Override
+    public void write(Output output)
+    {
+        CqlField.CqlType.write(this, output);
+        inner.write(output);
+    }
+
+    @Override
+    public String toString()
+    {
+        return cqlName();
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return hashCode;
+    }
+
+    @Override
+    public int compare(Object first, Object second)
+    {
+        return inner.compare(first, second);
+    }
+
+    @Override
+    public boolean equals(Object other)
+    {
+        if (other == null)
+        {
+            return false;
+        }
+        if (this == other)
+        {
+            return true;
+        }
+        if (this.getClass() != other.getClass())
+        {
+            return false;
+        }
+
+        CqlFrozen that = (CqlFrozen) other;
+        return new EqualsBuilder()
+               .append(this.internalType(), that.internalType())
+               .append(this.inner, that.inner)
+               .isEquals();
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/complex/CqlList.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/complex/CqlList.java
new file mode 100644
index 0000000..5970582
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/complex/CqlList.java
@@ -0,0 +1,180 @@
+/*
+ * 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.cassandra.spark.data.complex;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import org.apache.cassandra.bridge.BigNumberConfig;
+import org.apache.cassandra.bridge.CassandraVersion;
+import org.apache.cassandra.cql3.functions.types.SettableByIndexData;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.ListType;
+import org.apache.cassandra.db.rows.BufferCell;
+import org.apache.cassandra.db.rows.CellPath;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.serializers.ListSerializer;
+import org.apache.cassandra.serializers.TypeSerializer;
+import org.apache.cassandra.spark.data.CqlField;
+import org.apache.cassandra.spark.data.CqlType;
+import org.apache.cassandra.spark.utils.RandomUtils;
+import org.apache.cassandra.utils.UUIDGen;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+import org.apache.spark.sql.catalyst.util.ArrayData;
+import org.apache.spark.sql.catalyst.util.GenericArrayData;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import scala.collection.mutable.AbstractSeq;
+
+import static org.apache.cassandra.spark.utils.ScalaConversionUtils.mutableSeqAsJavaList;
+
+public class CqlList extends CqlCollection implements CqlField.CqlList
+{
+    public CqlList(CqlField.CqlType type)
+    {
+        super(type);
+    }
+
+    @Override
+    public AbstractType<?> dataType(boolean isMultiCell)
+    {
+        return ListType.getInstance(((CqlType) type()).dataType(), isMultiCell);
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public Object toSparkSqlType(Object value, boolean isFrozen)
+    {
+        return ArrayData.toArrayData(((Collection<Object>) value).stream()
+                                                                 .map(element -> type().toSparkSqlType(element))
+                                                                 .toArray());
+    }
+
+    @Override
+    public InternalType internalType()
+    {
+        return InternalType.List;
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public <T> TypeSerializer<T> serializer()
+    {
+        return (TypeSerializer<T>) ListSerializer.getInstance(((CqlType) type()).serializer());
+    }
+
+    @Override
+    public boolean equals(Object first, Object second)
+    {
+        return CqlField.equalsArrays(((GenericArrayData) first).array(),
+                                     ((GenericArrayData) second).array(), position -> type());
+    }
+
+    @Override
+    public String name()
+    {
+        return "list";
+    }
+
+    @Override
+    public DataType sparkSqlType(BigNumberConfig bigNumberConfig)
+    {
+        return DataTypes.createArrayType(type().sparkSqlType(bigNumberConfig));
+    }
+
+    @Override
+    public int compare(Object first, Object second)
+    {
+        return CqlField.compareArrays(((GenericArrayData) first).array(),
+                                      ((GenericArrayData) second).array(), position -> type());
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public Object toTestRowType(Object value)
+    {
+        return mutableSeqAsJavaList((AbstractSeq<?>) value).stream()
+                     .map(element -> type().toTestRowType(element))
+                     .collect(Collectors.toList());
+    }
+
+    @Override
+    public void setInnerValue(SettableByIndexData<?> udtValue, int position, Object value)
+    {
+        udtValue.setList(position, (List<?>) value);
+    }
+
+    @Override
+    public Object sparkSqlRowValue(GenericInternalRow row, int position)
+    {
+        return Arrays.stream(row.getArray(position).array())
+                     .map(element -> type().toTestRowType(element))
+                     .collect(Collectors.toList());
+    }
+
+    @Override
+    public Object sparkSqlRowValue(Row row, int position)
+    {
+        return row.getList(position).stream()
+                .map(element -> type().toTestRowType(element))
+                .collect(Collectors.toList());
+    }
+
+    @Override
+    public Object randomValue(int minCollectionSize)
+    {
+        return IntStream.range(0, RandomUtils.RANDOM.nextInt(16) + minCollectionSize)
+                        .mapToObj(element -> type().randomValue(minCollectionSize))
+                        .collect(Collectors.toList());
+    }
+
+    @Override
+    public org.apache.cassandra.cql3.functions.types.DataType driverDataType(boolean isFrozen)
+    {
+        return org.apache.cassandra.cql3.functions.types.DataType.list(((CqlType) type()).driverDataType(isFrozen));
+    }
+
+    @Override
+    public Object convertForCqlWriter(Object value, CassandraVersion version)
+    {
+        return ((List<?>) value).stream()
+                                .map(element -> type().convertForCqlWriter(element, version))
+                                .collect(Collectors.toList());
+    }
+
+    @Override
+    public void addCell(org.apache.cassandra.db.rows.Row.Builder rowBuilder,
+                        ColumnMetadata column,
+                        long timestamp,
+                        Object value)
+    {
+        ((List<?>) value).stream()
+                         .map(element -> BufferCell.live(column,
+                                                         timestamp,
+                                                         type().serialize(element),
+                                                          CellPath.create(ByteBuffer.wrap(UUIDGen.getTimeUUIDBytes()))))
+                         .forEachOrdered(rowBuilder::addCell);
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/complex/CqlMap.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/complex/CqlMap.java
new file mode 100644
index 0000000..5556c67
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/complex/CqlMap.java
@@ -0,0 +1,215 @@
+/*
+ * 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.cassandra.spark.data.complex;
+
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import org.apache.cassandra.bridge.BigNumberConfig;
+import org.apache.cassandra.bridge.CassandraVersion;
+import org.apache.cassandra.cql3.functions.types.SettableByIndexData;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.MapType;
+import org.apache.cassandra.db.rows.BufferCell;
+import org.apache.cassandra.db.rows.CellPath;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.serializers.MapSerializer;
+import org.apache.cassandra.serializers.TypeSerializer;
+import org.apache.cassandra.spark.data.CqlField;
+import org.apache.cassandra.spark.data.CqlType;
+import org.apache.cassandra.spark.utils.RandomUtils;
+import org.apache.cassandra.utils.Pair;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+import org.apache.spark.sql.catalyst.util.ArrayBasedMapData;
+import org.apache.spark.sql.catalyst.util.ArrayData;
+import org.apache.spark.sql.catalyst.util.MapData;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import scala.collection.JavaConverters;
+
+public class CqlMap extends CqlCollection implements CqlField.CqlMap
+{
+    public CqlMap(CqlField.CqlType keyType, CqlField.CqlType valueType)
+    {
+        super(keyType, valueType);
+    }
+
+    public CqlField.CqlType keyType()
+    {
+        return type();
+    }
+
+    public CqlField.CqlType valueType()
+    {
+        return type(1);
+    }
+
+    @Override
+    public AbstractType<?> dataType(boolean isMultiCell)
+    {
+        return MapType.getInstance(((CqlType) keyType()).dataType(), ((CqlType) valueType()).dataType(), isMultiCell);
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public Object toSparkSqlType(Object value, boolean isFrozen)
+    {
+        return mapToSparkSqlType((Map<Object, Object>) value);
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public <T> TypeSerializer<T> serializer()
+    {
+        return (TypeSerializer<T>) MapSerializer.getInstance(((CqlType) keyType()).serializer(),
+                                                             ((CqlType) valueType()).serializer(),
+                                                             ((CqlType) keyType()).dataType().comparatorSet);
+    }
+
+    @Override
+    public boolean equals(Object first, Object second)
+    {
+        return CqlField.equalsArrays(((MapData) first).valueArray().array(),
+                                     ((MapData) second).valueArray().array(), position -> valueType());
+    }
+
+    private ArrayBasedMapData mapToSparkSqlType(Map<Object, Object> map)
+    {
+        Object[] keys = new Object[map.size()];
+        Object[] values = new Object[map.size()];
+        int position = 0;
+        for (Map.Entry<Object, Object> entry : map.entrySet())
+        {
+            keys[position] = keyType().toSparkSqlType(entry.getKey());
+            values[position] = valueType().toSparkSqlType(entry.getValue());
+            position++;
+        }
+        return new ArrayBasedMapData(ArrayData.toArrayData(keys), ArrayData.toArrayData(values));
+    }
+
+    @Override
+    public InternalType internalType()
+    {
+        return InternalType.Map;
+    }
+
+    @Override
+    public String name()
+    {
+        return "map";
+    }
+
+    @Override
+    public DataType sparkSqlType(BigNumberConfig bigNumberConfig)
+    {
+        return DataTypes.createMapType(keyType().sparkSqlType(bigNumberConfig),
+                                       valueType().sparkSqlType(bigNumberConfig));
+    }
+
+    @Override
+    public Object sparkSqlRowValue(GenericInternalRow row, int position)
+    {
+        MapData map = row.getMap(position);
+        ArrayData keys = map.keyArray();
+        ArrayData values = map.valueArray();
+        Map<Object, Object> result = new LinkedHashMap<>(keys.numElements());
+        for (int element = 0; element < keys.numElements(); element++)
+        {
+            Object key = keyType().toTestRowType(keys.get(element, keyType().sparkSqlType()));
+            Object value = valueType().toTestRowType(values.get(element, valueType().sparkSqlType()));
+            result.put(key, value);
+        }
+        return result;
+    }
+
+    @Override
+    public Object sparkSqlRowValue(Row row, int position)
+    {
+        return row.getJavaMap(position).entrySet().stream()
+                .collect(Collectors.toMap(element -> keyType().toTestRowType(element.getKey()),
+                                          element -> valueType().toTestRowType(element.getValue())));
+    }
+
+    @Override
+    public int compare(Object first, Object second)
+    {
+        return CqlField.compareArrays(((MapData) first).valueArray().array(),
+                                      ((MapData) second).valueArray().array(), position -> valueType());
+    }
+
+    @Override
+    public Object randomValue(int minCollectionSize)
+    {
+        return IntStream.range(0, RandomUtils.RANDOM.nextInt(16) + minCollectionSize)
+                        .mapToObj(entry -> Pair.create(keyType().randomValue(minCollectionSize),
+                                                       valueType().randomValue(minCollectionSize)))
+                        .collect(Collectors.toMap(Pair::left, Pair::right, (first, second) -> first));
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public Object toTestRowType(Object value)
+    {
+        return ((Map<Object, Object>) JavaConverters.mapAsJavaMapConverter(((scala.collection.immutable.Map<?, ?>) value))
+                                                    .asJava()).entrySet().stream()
+                .collect(Collectors.toMap(element -> keyType().toTestRowType(element.getKey()),
+                                          element -> valueType().toTestRowType(element.getValue())));
+    }
+
+    @Override
+    public void setInnerValue(SettableByIndexData<?> udtValue, int position, Object value)
+    {
+        udtValue.setMap(position, (Map<?, ?>) value);
+    }
+
+    @Override
+    public org.apache.cassandra.cql3.functions.types.DataType driverDataType(boolean isFrozen)
+    {
+        return org.apache.cassandra.cql3.functions.types.DataType.map(((CqlType) keyType()).driverDataType(isFrozen),
+                                                                      ((CqlType) valueType()).driverDataType(isFrozen));
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public Object convertForCqlWriter(Object value, CassandraVersion version)
+    {
+        Map<Object, Object> map = (Map<Object, Object>) value;
+        return map.entrySet().stream()
+                .collect(Collectors.toMap(element -> keyType().convertForCqlWriter(element.getKey(), version),
+                                          element -> valueType().convertForCqlWriter(element.getValue(), version)));
+    }
+
+    @Override
+    public void addCell(org.apache.cassandra.db.rows.Row.Builder rowBuilder,
+                        ColumnMetadata column,
+                        long timestamp,
+                        Object value)
+    {
+        ((Map<?, ?>) value).entrySet().stream()
+                .map(element -> BufferCell.live(column,
+                                                timestamp,
+                                                valueType().serialize(element.getValue()),
+                                                CellPath.create(keyType().serialize(element.getKey()))))
+                .forEachOrdered(rowBuilder::addCell);
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/complex/CqlSet.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/complex/CqlSet.java
new file mode 100644
index 0000000..c824554
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/complex/CqlSet.java
@@ -0,0 +1,133 @@
+/*
+ * 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.cassandra.spark.data.complex;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.cassandra.bridge.CassandraVersion;
+import org.apache.cassandra.cql3.functions.types.DataType;
+import org.apache.cassandra.cql3.functions.types.SettableByIndexData;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.SetType;
+import org.apache.cassandra.db.rows.BufferCell;
+import org.apache.cassandra.db.rows.CellPath;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.serializers.SetSerializer;
+import org.apache.cassandra.serializers.TypeSerializer;
+import org.apache.cassandra.spark.data.CqlField;
+import org.apache.cassandra.spark.data.CqlType;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+
+@SuppressWarnings("unchecked")
+public class CqlSet extends CqlList implements CqlField.CqlSet
+{
+    public CqlSet(CqlField.CqlType type)
+    {
+        super(type);
+    }
+
+    @Override
+    public AbstractType<?> dataType(boolean isMultiCell)
+    {
+        return SetType.getInstance(((CqlType) type()).dataType(), isMultiCell);
+    }
+
+    @Override
+    public InternalType internalType()
+    {
+        return InternalType.Set;
+    }
+
+    @Override
+    public <T> TypeSerializer<T> serializer()
+    {
+        return (TypeSerializer<T>) SetSerializer.getInstance(((CqlType) type()).serializer(),
+                                                             ((CqlType) type()).dataType().comparatorSet);
+    }
+
+    @Override
+    public String name()
+    {
+        return "set";
+    }
+
+    @Override
+    public Object randomValue(int minCollectionSize)
+    {
+        return new HashSet<>(((List<Object>) super.randomValue(minCollectionSize)));
+    }
+
+    @Override
+    public Object toTestRowType(Object value)
+    {
+        return new HashSet<>(((List<Object>) super.toTestRowType(value)));
+    }
+
+    @Override
+    public Object sparkSqlRowValue(GenericInternalRow row, int position)
+    {
+        return new HashSet<>(((List<Object>) super.sparkSqlRowValue(row, position)));
+    }
+
+    @Override
+    public Object sparkSqlRowValue(Row row, int position)
+    {
+        return new HashSet<>(((List<Object>) super.sparkSqlRowValue(row, position)));
+    }
+
+    @Override
+    public void setInnerValue(SettableByIndexData<?> udtValue, int position, Object value)
+    {
+        udtValue.setSet(position, (Set<?>) value);
+    }
+
+    @Override
+    public DataType driverDataType(boolean isFrozen)
+    {
+        return DataType.set(((CqlType) type()).driverDataType(isFrozen));
+    }
+
+    @Override
+    public Object convertForCqlWriter(Object value, CassandraVersion version)
+    {
+        return ((Set<?>) value).stream()
+                               .map(element -> type().convertForCqlWriter(element, version))
+                               .collect(Collectors.toSet());
+    }
+
+    @Override
+    public void addCell(org.apache.cassandra.db.rows.Row.Builder rowBuilder,
+                        ColumnMetadata column,
+                        long timestamp,
+                        Object value)
+    {
+        ((Set<?>) value).stream()
+                        .map(element -> BufferCell.live(column,
+                                                        timestamp,
+                                                        ByteBufferUtil.EMPTY_BYTE_BUFFER,
+                                                        CellPath.create(type().serialize(element))))
+                        .forEachOrdered(rowBuilder::addCell);
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/complex/CqlTuple.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/complex/CqlTuple.java
new file mode 100644
index 0000000..03e6ea4
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/complex/CqlTuple.java
@@ -0,0 +1,244 @@
+/*
+ * 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.cassandra.spark.data.complex;
+
+import java.nio.Buffer;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import org.apache.cassandra.bridge.BigNumberConfig;
+import org.apache.cassandra.bridge.CassandraVersion;
+import org.apache.cassandra.cql3.functions.types.SettableByIndexData;
+import org.apache.cassandra.cql3.functions.types.TupleHelper;
+import org.apache.cassandra.cql3.functions.types.TupleValue;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.TupleType;
+import org.apache.cassandra.serializers.TupleSerializer;
+import org.apache.cassandra.serializers.TypeSerializer;
+import org.apache.cassandra.spark.data.CqlField;
+import org.apache.cassandra.spark.data.CqlType;
+import org.apache.cassandra.spark.utils.ByteBufferUtils;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.StructField;
+
+public class CqlTuple extends CqlCollection implements CqlField.CqlTuple
+{
+    CqlTuple(CqlField.CqlType... types)
+    {
+        super(types);
+    }
+
+    @Override
+    public AbstractType<?> dataType(boolean isMultiCell)
+    {
+        return new TupleType(types().stream()
+                                    .map(type -> (CqlType) type)
+                                    .map(CqlType::dataType)
+                                    .collect(Collectors.toList()));
+    }
+
+    @Override
+    public Object toSparkSqlType(Object value, boolean isFrozen)
+    {
+        if (value instanceof ByteBuffer)
+        {
+            // Need to deserialize first, e.g. if tuple is frozen inside collections
+            return deserialize((ByteBuffer) value);
+        }
+        else
+        {
+            return new GenericInternalRow((Object[]) value);
+        }
+    }
+
+    @Override
+    public ByteBuffer serialize(Object value)
+    {
+        return serializeTuple((Object[]) value);
+    }
+
+    @Override
+    public boolean equals(Object first, Object second)
+    {
+        return CqlField.equalsArrays(((GenericInternalRow) first).values(), ((GenericInternalRow) second).values(), this::type);
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public <T> TypeSerializer<T> serializer()
+    {
+        return (TypeSerializer<T>) new TupleSerializer(types().stream()
+                                                              .map(type -> (CqlType) type)
+                                                              .map(CqlType::serializer)
+                                                              .collect(Collectors.toList()));
+    }
+
+    @Override
+    public Object deserialize(ByteBuffer buffer, boolean isFrozen)
+    {
+        return toSparkSqlType(deserializeTuple(buffer, isFrozen));
+    }
+
+    @Override
+    public InternalType internalType()
+    {
+        return InternalType.Tuple;
+    }
+
+    @Override
+    public String name()
+    {
+        return "tuple";
+    }
+
+    @Override
+    public DataType sparkSqlType(BigNumberConfig bigNumberConfig)
+    {
+        return DataTypes.createStructType(IntStream.range(0, size())
+                                                   .mapToObj(index -> DataTypes.createStructField(
+                                                           Integer.toString(index),
+                                                           type(index).sparkSqlType(bigNumberConfig),
+                                                           true))
+                                                   .toArray(StructField[]::new));
+    }
+
+    @Override
+    public Object sparkSqlRowValue(GenericInternalRow row, int position)
+    {
+        InternalRow tupleStruct = row.getStruct(position, size());
+        return IntStream.range(0, size())
+                        .boxed()
+                        .map(index -> type(index).toTestRowType(tupleStruct.get(index, type(index).sparkSqlType())))
+                        .toArray();
+    }
+
+    @Override
+    public Object sparkSqlRowValue(Row row, int position)
+    {
+        Row tupleStruct = row.getStruct(position);
+        return IntStream.range(0, tupleStruct.size())
+                        .boxed()
+                        .filter(index -> !tupleStruct.isNullAt(index))
+                        .map(index -> type(index).toTestRowType(tupleStruct.get(index)))
+                        .toArray();
+    }
+
+    @Override
+    public ByteBuffer serializeTuple(Object[] values)
+    {
+        List<ByteBuffer> buffers = IntStream.range(0, size())
+                                            .mapToObj(index -> type(index).serialize(values[index]))
+                                            .collect(Collectors.toList());
+        ByteBuffer result = ByteBuffer.allocate(buffers.stream()
+                                                       .map(Buffer::remaining)
+                                                       .map(remaining -> remaining + 4)
+                                                       .reduce(Integer::sum)
+                                                       .orElse(0));
+        for (ByteBuffer buffer : buffers)
+        {
+            result.putInt(buffer.remaining());  // Length
+            result.put(buffer.duplicate());  // Value
+        }
+        // Cast to ByteBuffer required when compiling with Java 8
+        return (ByteBuffer) result.flip();
+    }
+
+    @Override
+    public Object[] deserializeTuple(ByteBuffer buffer, boolean isFrozen)
+    {
+        Object[] result = new Object[size()];
+        int position = 0;
+        for (CqlField.CqlType type : types())
+        {
+            if (buffer.remaining() < 4)
+            {
+                break;
+            }
+            int length = buffer.getInt();
+            result[position++] = length > 0 ? type.deserialize(ByteBufferUtils.readBytes(buffer, length), isFrozen) : null;
+        }
+        return result;
+    }
+
+    @Override
+    public int compare(Object first, Object second)
+    {
+        return CqlField.compareArrays(((GenericInternalRow) first).values(), ((GenericInternalRow) second).values(), this::type);
+    }
+
+    @Override
+    public Object toTestRowType(Object value)
+    {
+        GenericRowWithSchema tupleRow = (GenericRowWithSchema) value;
+        Object[] tupleResult = new Object[tupleRow.size()];
+        for (int index = 0; index < tupleRow.size(); index++)
+        {
+            tupleResult[index] = type(index).toTestRowType(tupleRow.get(index));
+        }
+        return tupleResult;
+    }
+
+    @Override
+    public void setInnerValue(SettableByIndexData<?> udtValue, int position, Object value)
+    {
+        udtValue.setTupleValue(position, toTupleValue(CassandraVersion.FOURZERO, this, value));
+    }
+
+    @Override
+    public Object randomValue(int minCollectionSize)
+    {
+        return types().stream().map(type -> type.randomValue(minCollectionSize)).toArray();
+    }
+
+    @Override
+    public org.apache.cassandra.cql3.functions.types.DataType driverDataType(boolean isFrozen)
+    {
+        return TupleHelper.buildTupleType(this, isFrozen);
+    }
+
+    @Override
+    public Object convertForCqlWriter(Object value, CassandraVersion version)
+    {
+        return toTupleValue(version, this, value);
+    }
+
+    public static TupleValue toTupleValue(CassandraVersion version, CqlTuple tuple, Object value)
+    {
+        if (value instanceof TupleValue)
+        {
+            return (TupleValue) value;
+        }
+
+        TupleValue tupleValue = TupleHelper.buildTupleValue(tuple);
+        Object[] array = (Object[]) value;
+        for (int position = 0; position < array.length; position++)
+        {
+            CqlUdt.setInnerValue(version, tupleValue, (CqlType) tuple.type(position), position, array[position]);
+        }
+        return tupleValue;
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/complex/CqlUdt.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/complex/CqlUdt.java
new file mode 100644
index 0000000..275742d
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/complex/CqlUdt.java
@@ -0,0 +1,535 @@
+/*
+ * 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.cassandra.spark.data.complex;
+
+import java.nio.Buffer;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.lang.builder.EqualsBuilder;
+import org.apache.commons.lang.builder.HashCodeBuilder;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+import org.apache.cassandra.bridge.BigNumberConfig;
+import org.apache.cassandra.bridge.CassandraBridge;
+import org.apache.cassandra.bridge.CassandraVersion;
+import org.apache.cassandra.cql3.functions.types.SettableByIndexData;
+import org.apache.cassandra.cql3.functions.types.UDTValue;
+import org.apache.cassandra.cql3.functions.types.UserType;
+import org.apache.cassandra.cql3.functions.types.UserTypeHelper;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.serializers.TypeSerializer;
+import org.apache.cassandra.serializers.UTF8Serializer;
+import org.apache.cassandra.spark.data.CqlField;
+import org.apache.cassandra.spark.data.CqlType;
+import org.apache.cassandra.spark.utils.ByteBufferUtils;
+import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.StructField;
+
+public class CqlUdt extends CqlType implements CqlField.CqlUdt
+{
+    private final String keyspace;
+    private final String name;
+    private final List<CqlField> fields;
+    private final Map<String, CqlField> fieldMap;
+    private final int hashCode;
+
+    CqlUdt(String keyspace, String name, List<CqlField> fields)
+    {
+        this.keyspace = keyspace;
+        this.name = name;
+        this.fields = Collections.unmodifiableList(fields);
+        this.fieldMap = this.fields.stream().collect(Collectors.toMap(CqlField::name, Function.identity()));
+        this.hashCode = new HashCodeBuilder()
+                .append(internalType().ordinal())
+                .append(this.keyspace)
+                .append(this.name)
+                .append(this.fields)
+                .toHashCode();
+    }
+
+    @Override
+    public Set<CqlField.CqlUdt> udts()
+    {
+        Set<CqlField.CqlUdt> udts = fields.stream()
+                                          .map(CqlField::type)
+                                          .map(type -> (CqlType) type)
+                                          .map(CqlField.CqlType::udts)
+                                          .flatMap(Collection::stream)
+                                          .collect(Collectors.toSet());
+        udts.add(this);
+        return udts;
+    }
+
+    @Override
+    public Object sparkSqlRowValue(GenericInternalRow row, int position)
+    {
+        InternalRow struct = row.getStruct(position, size());
+        return IntStream.range(0, size())
+                        .boxed()
+                        .collect(Collectors.toMap(index -> field(index).name(),
+                                                  index -> type(index).toTestRowType(struct.get(index, type(index).sparkSqlType()))));
+    }
+
+    @Override
+    public Object sparkSqlRowValue(Row row, int position)
+    {
+        Row struct = row.getStruct(position);
+        return IntStream.range(0, struct.size())
+                        .boxed()
+                        .filter(index -> !struct.isNullAt(index))
+                        .collect(Collectors.toMap(index -> struct.schema().fields()[index].name(),
+                                                  index -> field(index).type().toTestRowType(struct.get(index))));
+    }
+
+    @Override
+    public Object randomValue(int minCollectionSize)
+    {
+        return fields().stream()
+                       .collect(Collectors.toMap(CqlField::name, field -> Objects.requireNonNull(field.type().randomValue(minCollectionSize))));
+    }
+
+    @Override
+    public Object toTestRowType(Object value)
+    {
+        GenericRowWithSchema row = (GenericRowWithSchema) value;
+        String[] fieldNames = row.schema().fieldNames();
+        Map<String, Object> result = new LinkedHashMap<>(fieldNames.length);
+        for (int fieldName = 0; fieldName < fieldNames.length; fieldName++)
+        {
+            result.put(fieldNames[fieldName], field(fieldName).type().toTestRowType(row.get(fieldName)));
+        }
+        return result;
+    }
+
+    @Override
+    public void setInnerValue(SettableByIndexData<?> udtValue, int position, Object value)
+    {
+        udtValue.setUDTValue(position, (UDTValue) value);
+    }
+
+    @Override
+    public org.apache.cassandra.cql3.functions.types.DataType driverDataType(boolean isFrozen)
+    {
+        return UserTypeHelper.newUserType(
+                keyspace(),
+                name(),
+                isFrozen,
+                fields().stream()
+                        .map(field -> UserTypeHelper.newField(field.name(),
+                                                              ((CqlType) field.type()).driverDataType(isFrozen)))
+                        .collect(Collectors.toList()),
+                ProtocolVersion.V3);
+    }
+
+    @Override
+    public Object convertForCqlWriter(Object value, CassandraVersion version)
+    {
+        return toUserTypeValue(version, this, value);
+    }
+
+    @Override
+    public String toString()
+    {
+        return cqlName();
+    }
+
+    public CqlFrozen frozen()
+    {
+        return CqlFrozen.build(this);
+    }
+
+    public static Builder builder(String keyspace, String name)
+    {
+        return new Builder(keyspace, name);
+    }
+
+    public static class Builder implements CqlField.CqlUdtBuilder
+    {
+        private final String keyspace;
+        private final String name;
+        private final List<CqlField> fields = new ArrayList<>();
+
+        public Builder(String keyspace, String name)
+        {
+            this.keyspace = keyspace;
+            this.name = name;
+        }
+
+        @Override
+        public Builder withField(String name, CqlField.CqlType type)
+        {
+            fields.add(new CqlField(false, false, false, name, type, fields.size()));
+            return this;
+        }
+
+        @Override
+        public CqlUdt build()
+        {
+            return new CqlUdt(keyspace, name, fields);
+        }
+    }
+
+    @Override
+    public boolean isSupported()
+    {
+        return true;
+    }
+
+    @Override
+    public AbstractType<?> dataType()
+    {
+        return dataType(true);
+    }
+
+    @Override
+    public AbstractType<?> dataType(boolean isMultiCell)
+    {
+        // Get UserTypeSerializer from Schema instance to ensure fields are deserialized in correct order
+        return Schema.instance.getKeyspaceMetadata(keyspace()).types
+               .get(UTF8Serializer.instance.serialize(name()))
+               .orElseThrow(() -> new RuntimeException(String.format("UDT '%s' not initialized", name())));
+    }
+
+    @Override
+    public Object toSparkSqlType(Object value)
+    {
+        return toSparkSqlType(value, false);
+    }
+
+    @Override
+    public Object toSparkSqlType(Object value, boolean isFrozen)
+    {
+        return udtToSparkSqlType(value, isFrozen);
+    }
+
+    @SuppressWarnings("unchecked")
+    private GenericInternalRow udtToSparkSqlType(Object value, boolean isFrozen)
+    {
+        if (value instanceof ByteBuffer)
+        {
+            // Need to deserialize first, e.g. if UDT is frozen inside collections
+            return udtToSparkSqlType(deserializeUdt((ByteBuffer) value, isFrozen));
+        }
+        else
+        {
+            return udtToSparkSqlType((Map<String, Object>) value);
+        }
+    }
+
+    private GenericInternalRow udtToSparkSqlType(Map<String, Object> value)
+    {
+        Object[] objects = new Object[size()];
+        for (int index = 0; index < size(); index++)
+        {
+            objects[index] = value.getOrDefault(field(index).name(), null);
+        }
+        return new GenericInternalRow(objects);
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public <T> TypeSerializer<T> serializer()
+    {
+        // Get UserTypeSerializer from Schema instance to ensure fields are deserialized in correct order
+        return (TypeSerializer<T>) Schema.instance.getKeyspaceMetadata(keyspace()).types
+                .get(UTF8Serializer.instance.serialize(name()))
+                .orElseThrow(() -> new RuntimeException(String.format("UDT '%s' not initialized", name())))
+                .getSerializer();
+    }
+
+    @Override
+    public Object deserialize(ByteBuffer buffer)
+    {
+        return deserialize(buffer, false);
+    }
+
+    @Override
+    public Object deserialize(ByteBuffer buffer, boolean isFrozen)
+    {
+        return udtToSparkSqlType(deserializeUdt(buffer, isFrozen));
+    }
+
+    @Override
+    public Map<String, Object> deserializeUdt(ByteBuffer buffer, boolean isFrozen)
+    {
+        if (!isFrozen)
+        {
+            int fieldCount = buffer.getInt();
+            Preconditions.checkArgument(fieldCount == size(),
+                    String.format("Unexpected number of fields deserializing UDT '%s', expected %d fields but %d found",
+                                  cqlName(), size(), fieldCount));
+        }
+
+        Map<String, Object> result = new LinkedHashMap<>(size());
+        for (CqlField field : fields())
+        {
+            if (buffer.remaining() < 4)
+            {
+                break;
+            }
+            int length = buffer.getInt();
+            result.put(field.name(), length > 0 ? field.deserialize(ByteBufferUtils.readBytes(buffer, length), isFrozen) : null);
+        }
+
+        return result;
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public ByteBuffer serialize(Object value)
+    {
+        return serializeUdt((Map<String, Object>) value);
+    }
+
+    @Override
+    public ByteBuffer serializeUdt(Map<String, Object> values)
+    {
+        List<ByteBuffer> buffers = fields().stream()
+                                           .map(field -> field.serialize(values.get(field.name())))
+                                           .collect(Collectors.toList());
+
+        ByteBuffer result = ByteBuffer.allocate(4 + buffers.stream()
+                                                           .map(Buffer::remaining)
+                                                           .map(remaining -> remaining + 4)
+                                                           .reduce(Integer::sum)
+                                                           .orElse(0));
+        result.putInt(buffers.size());  // Number of fields
+        for (ByteBuffer buffer : buffers)
+        {
+            result.putInt(buffer.remaining());  // Length
+            result.put(buffer.duplicate());  // Value
+        }
+        // Cast to ByteBuffer required when compiling with Java 8
+        return (ByteBuffer) result.flip();
+    }
+
+    @Override
+    public boolean equals(Object first, Object second)
+    {
+        return CqlField.equalsArrays(((GenericInternalRow) first).values(), ((GenericInternalRow) second).values(), this::type);
+    }
+
+    public InternalType internalType()
+    {
+        return InternalType.Udt;
+    }
+
+    public String createStatement(String keyspace)
+    {
+        return String.format("CREATE TYPE %s.%s (%s);", keyspace, name, fieldsString());
+    }
+
+    private String fieldsString()
+    {
+        return fields.stream().map(CqlUdt::fieldString).collect(Collectors.joining(", "));
+    }
+
+    private static String fieldString(CqlField field)
+    {
+        return String.format("%s %s", field.name(), field.type().cqlName());
+    }
+
+    public String keyspace()
+    {
+        return keyspace;
+    }
+
+    public String name()
+    {
+        return name;
+    }
+
+    public int size()
+    {
+        return fields.size();
+    }
+
+    public List<CqlField> fields()
+    {
+        return fields;
+    }
+
+    public CqlField field(String name)
+    {
+        return fieldMap.get(name);
+    }
+
+    public CqlField field(int position)
+    {
+        return fields.get(position);
+    }
+
+    public CqlField.CqlType type(int position)
+    {
+        return field(position).type();
+    }
+
+    public String cqlName()
+    {
+        return name;
+    }
+
+    @Override
+    public DataType sparkSqlType(BigNumberConfig bigNumberConfig)
+    {
+        return DataTypes.createStructType(fields().stream()
+                .map(field -> DataTypes.createStructField(field.name(),
+                                                          field.type().sparkSqlType(bigNumberConfig),
+                                                          true))
+                .toArray(StructField[]::new));
+    }
+
+    public static CqlUdt read(Input input, CassandraBridge bridge)
+    {
+        Builder builder = CqlUdt.builder(input.readString(), input.readString());
+        int numFields = input.readInt();
+        for (int field = 0; field < numFields; field++)
+        {
+            builder.withField(input.readString(), CqlField.CqlType.read(input, bridge));
+        }
+        return builder.build();
+    }
+
+    @Override
+    public void write(Output output)
+    {
+        CqlField.CqlType.write(this, output);
+        output.writeString(this.keyspace);
+        output.writeString(this.name);
+        output.writeInt(this.fields.size());
+        for (CqlField field : this.fields)
+        {
+            output.writeString(field.name());
+            field.type().write(output);
+        }
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return hashCode;
+    }
+
+    @Override
+    public int compare(Object first, Object second)
+    {
+        return CqlField.compareArrays(((GenericInternalRow) first).values(), ((GenericInternalRow) second).values(), this::type);
+    }
+
+    @Override
+    public boolean equals(Object other)
+    {
+        if (other == null)
+        {
+            return false;
+        }
+        if (this == other)
+        {
+            return true;
+        }
+        if (this.getClass() != other.getClass())
+        {
+            return false;
+        }
+
+        CqlUdt that = (CqlUdt) other;
+        return new EqualsBuilder()
+               .append(this.internalType(), that.internalType())
+               .append(this.keyspace, that.keyspace)
+               .append(this.name, that.name)
+               .append(this.fields, that.fields)
+               .isEquals();
+    }
+
+    public static class Serializer extends com.esotericsoftware.kryo.Serializer<CqlUdt>
+    {
+        private final CassandraBridge bridge;
+
+        public Serializer(CassandraBridge bridge)
+        {
+            this.bridge = bridge;
+        }
+
+        @Override
+        public CqlUdt read(Kryo kryo, Input input, Class type)
+        {
+            return CqlUdt.read(input, bridge);
+        }
+
+        @Override
+        public void write(Kryo kryo, Output output, CqlUdt udt)
+        {
+            udt.write(output);
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    public static UDTValue toUserTypeValue(CassandraVersion version, CqlUdt udt, Object value)
+    {
+        Map<String, Object> values = (Map<String, Object>) value;
+        UDTValue udtValue = UserTypeHelper.newUDTValue(toUserType(udt));
+        int position = 0;
+        for (CqlField field : udt.fields())
+        {
+            setInnerValue(version, udtValue, (CqlType) field.type(), position++, values.get(field.name()));
+        }
+        return udtValue;
+    }
+
+    // Set inner value for UDTs or Tuples
+    public static void setInnerValue(CassandraVersion version,
+                                     SettableByIndexData<?> udtValue,
+                                     CqlType type,
+                                     int position,
+                                     Object value)
+    {
+        type.setInnerValue(udtValue, position, type.convertForCqlWriter(value, version));
+    }
+
+    public static UserType toUserType(CqlUdt udt)
+    {
+        List<UserType.Field> fields = udt.fields().stream()
+                .map(field -> UserTypeHelper.newField(field.name(),
+                                                      ((CqlType) field.type()).driverDataType()))
+                .collect(Collectors.toList());
+        return UserTypeHelper.newUserType(udt.keyspace(), udt.name(), true, fields, ProtocolVersion.V3);
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/Ascii.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/Ascii.java
new file mode 100644
index 0000000..3810efe
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/Ascii.java
@@ -0,0 +1,47 @@
+/*
+ * 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.cassandra.spark.data.types;
+
+import org.apache.cassandra.cql3.functions.types.DataType;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.AsciiType;
+
+public class Ascii extends StringBased
+{
+    public static final Ascii INSTANCE = new Ascii();
+
+    @Override
+    public String name()
+    {
+        return "ascii";
+    }
+
+    @Override
+    public AbstractType<?> dataType()
+    {
+        return AsciiType.instance;
+    }
+
+    @Override
+    public DataType driverDataType(boolean isFrozen)
+    {
+        return DataType.ascii();
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/BigInt.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/BigInt.java
new file mode 100644
index 0000000..86f998a
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/BigInt.java
@@ -0,0 +1,54 @@
+/*
+ * 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.cassandra.spark.data.types;
+
+import org.apache.cassandra.cql3.functions.types.DataType;
+import org.apache.cassandra.cql3.functions.types.SettableByIndexData;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.LongType;
+
+public class BigInt extends LongBased
+{
+    public static final BigInt INSTANCE = new BigInt();
+
+    @Override
+    public String name()
+    {
+        return "bigint";
+    }
+
+    @Override
+    public AbstractType<?> dataType()
+    {
+        return LongType.instance;
+    }
+
+    @Override
+    public void setInnerValue(SettableByIndexData<?> udtValue, int position, Object value)
+    {
+        udtValue.setLong(position, (long) value);
+    }
+
+    @Override
+    public DataType driverDataType(boolean isFrozen)
+    {
+        return DataType.bigint();
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/BinaryBased.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/BinaryBased.java
new file mode 100644
index 0000000..4e1a35f
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/BinaryBased.java
@@ -0,0 +1,60 @@
+/*
+ * 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.cassandra.spark.data.types;
+
+import java.util.Comparator;
+
+import com.google.common.primitives.UnsignedBytes;
+
+import org.apache.cassandra.bridge.BigNumberConfig;
+import org.apache.cassandra.spark.data.NativeType;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+
+public abstract class BinaryBased extends NativeType
+{
+    public static final Comparator<byte[]> BYTE_ARRAY_COMPARATOR = UnsignedBytes.lexicographicalComparator();
+
+    @Override
+    protected int compareTo(Object first, Object second)
+    {
+        return BYTE_ARRAY_COMPARATOR.compare((byte[]) first, (byte[]) second);
+    }
+
+    @Override
+    public DataType sparkSqlType(BigNumberConfig bigNumberConfig)
+    {
+        return DataTypes.BinaryType;
+    }
+
+    @Override
+    protected Object nativeSparkSqlRowValue(GenericInternalRow row, int position)
+    {
+        return row.getBinary(position);
+    }
+
+    @Override
+    protected Object nativeSparkSqlRowValue(Row row, int position)
+    {
+        return row.getAs(position);
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/Blob.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/Blob.java
new file mode 100644
index 0000000..a74251c
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/Blob.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.cassandra.spark.data.types;
+
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.cql3.functions.types.DataType;
+import org.apache.cassandra.cql3.functions.types.SettableByIndexData;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.spark.utils.ByteBufferUtils;
+import org.apache.cassandra.spark.utils.RandomUtils;
+
+public class Blob extends BinaryBased
+{
+    public static final Blob INSTANCE = new Blob();
+
+    @Override
+    public String name()
+    {
+        return "blob";
+    }
+
+    @Override
+    public AbstractType<?> dataType()
+    {
+        return BytesType.instance;
+    }
+
+    @Override
+    public Object toSparkSqlType(Object value, boolean isFrozen)
+    {
+        return ByteBufferUtils.getArray((ByteBuffer) value);
+    }
+
+    @Override
+    public Object randomValue(int minCollectionSize)
+    {
+        return RandomUtils.randomByteBuffer(RandomUtils.randomPositiveInt(256));
+    }
+
+    @Override
+    public Object toTestRowType(Object value)
+    {
+        return ByteBuffer.wrap((byte[]) value);
+    }
+
+    @Override
+    public void setInnerValue(SettableByIndexData<?> udtValue, int position, Object value)
+    {
+        udtValue.setBytes(position, (ByteBuffer) value);
+    }
+
+    @Override
+    public DataType driverDataType(boolean isFrozen)
+    {
+        return DataType.blob();
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/Boolean.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/Boolean.java
new file mode 100644
index 0000000..0bc5e92
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/Boolean.java
@@ -0,0 +1,99 @@
+/*
+ * 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.cassandra.spark.data.types;
+
+import java.util.Comparator;
+
+import org.apache.cassandra.bridge.BigNumberConfig;
+import org.apache.cassandra.cql3.functions.types.SettableByIndexData;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.BooleanType;
+import org.apache.cassandra.spark.data.NativeType;
+import org.apache.cassandra.spark.utils.RandomUtils;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+
+public class Boolean extends NativeType
+{
+    public static final Boolean INSTANCE = new Boolean();
+    private static final Comparator<java.lang.Boolean> BOOLEAN_COMPARATOR = java.lang.Boolean::compareTo;
+
+    @Override
+    public String name()
+    {
+        return "boolean";
+    }
+
+    @Override
+    public DataType sparkSqlType(BigNumberConfig bigNumberConfig)
+    {
+        return DataTypes.BooleanType;
+    }
+
+    @Override
+    public AbstractType<?> dataType()
+    {
+        return BooleanType.instance;
+    }
+
+    @Override
+    protected int compareTo(Object first, Object second)
+    {
+        return BOOLEAN_COMPARATOR.compare((java.lang.Boolean) first, (java.lang.Boolean) second);
+    }
+
+    @Override
+    public int cardinality(int orElse)
+    {
+        return 2;
+    }
+
+    @Override
+    protected Object nativeSparkSqlRowValue(GenericInternalRow row, int position)
+    {
+        return row.getBoolean(position);
+    }
+
+    @Override
+    protected Object nativeSparkSqlRowValue(Row row, int position)
+    {
+        return row.getBoolean(position);
+    }
+
+    @Override
+    public Object randomValue(int minCollectionSize)
+    {
+        return RandomUtils.RANDOM.nextBoolean();
+    }
+
+    @Override
+    public void setInnerValue(SettableByIndexData<?> udtValue, int position, Object value)
+    {
+        udtValue.setBool(position, (boolean) value);
+    }
+
+    @Override
+    public org.apache.cassandra.cql3.functions.types.DataType driverDataType(boolean isFrozen)
+    {
+        return org.apache.cassandra.cql3.functions.types.DataType.cboolean();
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/Counter.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/Counter.java
new file mode 100644
index 0000000..fa4875a
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/Counter.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.cassandra.spark.data.types;
+
+import org.apache.cassandra.spark.data.NativeType;
+
+public class Counter extends NativeType
+{
+    public static final Counter INSTANCE = new Counter();
+
+    @Override
+    public String name()
+    {
+        return "counter";
+    }
+
+    @Override
+    public boolean isSupported()
+    {
+        return false;
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/Date.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/Date.java
new file mode 100644
index 0000000..f6a3c19
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/Date.java
@@ -0,0 +1,129 @@
+/*
+ * 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.cassandra.spark.data.types;
+
+import org.apache.cassandra.bridge.BigNumberConfig;
+import org.apache.cassandra.bridge.CassandraVersion;
+import org.apache.cassandra.cql3.functions.types.LocalDate;
+import org.apache.cassandra.cql3.functions.types.SettableByIndexData;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.SimpleDateType;
+import org.apache.cassandra.spark.data.NativeType;
+import org.apache.cassandra.spark.utils.RandomUtils;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.jetbrains.annotations.NotNull;
+
+public class Date extends NativeType
+{
+    public static final Date INSTANCE = new Date();
+
+    @Override
+    public String name()
+    {
+        return "date";
+    }
+
+    @Override
+    public DataType sparkSqlType(BigNumberConfig bigNumberConfig)
+    {
+        return DataTypes.DateType;
+    }
+
+    @Override
+    public Object toSparkSqlType(@NotNull Object value, boolean isFrozen)
+    {
+        // SparkSQL date type is an int incrementing from day 0 on 1970-01-01
+        // Cassandra stores date as "days since 1970-01-01 plus Integer.MIN_VALUE"
+        int days = (Integer) value;
+        return days - Integer.MIN_VALUE;
+    }
+
+    @Override
+    public AbstractType<?> dataType()
+    {
+        return SimpleDateType.instance;
+    }
+
+    @Override
+    protected int compareTo(Object first, Object second)
+    {
+        return Int.INTEGER_COMPARATOR.compare((Integer) first, (Integer) second);
+    }
+
+    @Override
+    protected Object nativeSparkSqlRowValue(GenericInternalRow row, int position)
+    {
+        return row.getInt(position);
+    }
+
+    @Override
+    protected Object nativeSparkSqlRowValue(Row row, int position)
+    {
+        return row.getDate(position);
+    }
+
+    @Override
+    public Object toTestRowType(Object value)
+    {
+        if (value instanceof java.sql.Date)
+        {
+            // Round up to convert date back to days since epoch
+            return (int) ((java.sql.Date) value).toLocalDate().toEpochDay();
+        }
+        else if (value instanceof Integer)
+        {
+            return ((Integer) value) - Integer.MIN_VALUE;
+        }
+        return value;
+    }
+
+    @Override
+    public Object randomValue(int minCollectionSize)
+    {
+        return RandomUtils.randomPositiveInt(30_000);
+    }
+
+    @Override
+    public void setInnerValue(SettableByIndexData<?> udtValue, int position, Object value)
+    {
+        udtValue.setDate(position, (LocalDate) value);
+    }
+
+    @Override
+    public org.apache.cassandra.cql3.functions.types.DataType driverDataType(boolean isFrozen)
+    {
+        return org.apache.cassandra.cql3.functions.types.DataType.date();
+    }
+
+    @Override
+    public Object convertForCqlWriter(Object value, CassandraVersion version)
+    {
+        // Cassandra 4.0 no longer allows writing date types as Integers in CqlWriter,
+        // so we need to convert to LocalDate before writing in tests
+        if (version == CassandraVersion.FOURZERO)
+        {
+            return LocalDate.fromDaysSinceEpoch(((int) value));
+        }
+        return value;
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/Decimal.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/Decimal.java
new file mode 100644
index 0000000..3e309a7
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/Decimal.java
@@ -0,0 +1,113 @@
+/*
+ * 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.cassandra.spark.data.types;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.util.Comparator;
+
+import org.apache.cassandra.bridge.BigNumberConfig;
+import org.apache.cassandra.cql3.functions.types.SettableByIndexData;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.DecimalType;
+import org.apache.cassandra.spark.data.NativeType;
+import org.apache.cassandra.spark.utils.RandomUtils;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+
+public class Decimal extends NativeType
+{
+    public static final Decimal INSTANCE = new Decimal();
+    private static final Comparator<org.apache.spark.sql.types.Decimal> DECIMAL_COMPARATOR = Comparator.naturalOrder();
+
+    @Override
+    public String name()
+    {
+        return "decimal";
+    }
+
+    @Override
+    public DataType sparkSqlType(BigNumberConfig bigNumberConfig)
+    {
+        return DataTypes.createDecimalType(bigNumberConfig.bigDecimalPrecision(), bigNumberConfig.bigDecimalScale());
+    }
+
+    @Override
+    public AbstractType<?> dataType()
+    {
+        return DecimalType.instance;
+    }
+
+    @Override
+    public Object toSparkSqlType(Object value, boolean isFrozen)
+    {
+        return org.apache.spark.sql.types.Decimal.apply((BigDecimal) value);
+    }
+
+    @Override
+    protected int compareTo(Object first, Object second)
+    {
+        return DECIMAL_COMPARATOR.compare((org.apache.spark.sql.types.Decimal) first, (org.apache.spark.sql.types.Decimal) second);
+    }
+
+    @Override
+    protected Object nativeSparkSqlRowValue(GenericInternalRow row, int position)
+    {
+        return row.getDecimal(position, BigNumberConfig.DEFAULT.bigIntegerPrecision(), BigNumberConfig.DEFAULT.bigIntegerScale());
+    }
+
+    @Override
+    protected Object nativeSparkSqlRowValue(Row row, int position)
+    {
+        return row.getDecimal(position);
+    }
+
+    @Override
+    public Object toTestRowType(Object value)
+    {
+        if (value instanceof BigDecimal)
+        {
+            return value;
+        }
+        return ((org.apache.spark.sql.types.Decimal) value).toJavaBigDecimal();
+    }
+
+    @Override
+    public Object randomValue(int minCollectionSize)
+    {
+        BigInteger unscaledVal = new BigInteger(BigNumberConfig.DEFAULT.bigDecimalPrecision(), RandomUtils.RANDOM);
+        int scale = RandomUtils.RANDOM.nextInt(BigNumberConfig.DEFAULT.bigDecimalScale());
+        return new BigDecimal(unscaledVal, scale);
+    }
+
+    @Override
+    public void setInnerValue(SettableByIndexData<?> udtValue, int position, Object value)
+    {
+        udtValue.setDecimal(position, (BigDecimal) value);
+    }
+
+    @Override
+    public org.apache.cassandra.cql3.functions.types.DataType driverDataType(boolean isFrozen)
+    {
+        return org.apache.cassandra.cql3.functions.types.DataType.decimal();
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/Double.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/Double.java
new file mode 100644
index 0000000..7c522fd
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/Double.java
@@ -0,0 +1,93 @@
+/*
+ * 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.cassandra.spark.data.types;
+
+import java.util.Comparator;
+
+import org.apache.cassandra.bridge.BigNumberConfig;
+import org.apache.cassandra.cql3.functions.types.SettableByIndexData;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.DoubleType;
+import org.apache.cassandra.spark.data.NativeType;
+import org.apache.cassandra.spark.utils.RandomUtils;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+
+public class Double extends NativeType
+{
+    public static final Double INSTANCE = new Double();
+    private static final Comparator<java.lang.Double> DOUBLE_COMPARATOR = java.lang.Double::compareTo;
+
+    @Override
+    public String name()
+    {
+        return "double";
+    }
+
+    @Override
+    public DataType sparkSqlType(BigNumberConfig bigNumberConfig)
+    {
+        return DataTypes.DoubleType;
+    }
+
+    @Override
+    public AbstractType<?> dataType()
+    {
+        return DoubleType.instance;
+    }
+
+    @Override
+    protected int compareTo(Object first, Object second)
+    {
+        return DOUBLE_COMPARATOR.compare((java.lang.Double) first, (java.lang.Double) second);
+    }
+
+    @Override
+    protected Object nativeSparkSqlRowValue(GenericInternalRow row, int position)
+    {
+        return row.getDouble(position);
+    }
+
+    @Override
+    protected Object nativeSparkSqlRowValue(Row row, int position)
+    {
+        return row.getDouble(position);
+    }
+
+    @Override
+    public Object randomValue(int minCollectionSize)
+    {
+        return RandomUtils.RANDOM.nextDouble();
+    }
+
+    @Override
+    public void setInnerValue(SettableByIndexData<?> udtValue, int position, Object value)
+    {
+        udtValue.setDouble(position, (double) value);
+    }
+
+    @Override
+    public org.apache.cassandra.cql3.functions.types.DataType driverDataType(boolean isFrozen)
+    {
+        return org.apache.cassandra.cql3.functions.types.DataType.cdouble();
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/Duration.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/Duration.java
new file mode 100644
index 0000000..1e8ca98
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/Duration.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.cassandra.spark.data.types;
+
+import org.apache.cassandra.spark.data.NativeType;
+
+public class Duration extends NativeType
+{
+    public static final Duration INSTANCE = new Duration();
+
+    @Override
+    public String name()
+    {
+        return "duration";
+    }
+
+    @Override
+    public boolean isSupported()
+    {
+        return false;
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/Empty.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/Empty.java
new file mode 100644
index 0000000..3b34780
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/Empty.java
@@ -0,0 +1,98 @@
+/*
+ * 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.cassandra.spark.data.types;
+
+import java.util.Comparator;
+
+import org.apache.cassandra.bridge.BigNumberConfig;
+import org.apache.cassandra.cql3.functions.types.SettableByIndexData;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.EmptyType;
+import org.apache.cassandra.spark.data.NativeType;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+
+public class Empty extends NativeType
+{
+    public static final Empty INSTANCE = new Empty();
+    private static final Comparator<Void> VOID_COMPARATOR_COMPARATOR = (first, second) -> 0;
+
+    @Override
+    public boolean isSupported()
+    {
+        return false;
+    }
+
+    @Override
+    public String name()
+    {
+        return "empty";
+    }
+
+    @Override
+    public DataType sparkSqlType(BigNumberConfig bigNumberConfig)
+    {
+        return DataTypes.NullType;
+    }
+
+    @Override
+    public AbstractType<?> dataType()
+    {
+        return EmptyType.instance;
+    }
+
+    @Override
+    protected int compareTo(Object first, Object second)
+    {
+        return VOID_COMPARATOR_COMPARATOR.compare((Void) first, (Void) second);
+    }
+
+    @Override
+    public int cardinality(int orElse)
+    {
+        return 1;
+    }
+
+    @Override
+    protected Object nativeSparkSqlRowValue(GenericInternalRow row, int position)
+    {
+        return null;
+    }
+
+    @Override
+    protected Object nativeSparkSqlRowValue(Row row, int position)
+    {
+        return null;
+    }
+
+    @Override
+    public Object randomValue(int minCollectionSize)
+    {
+        return null;
+    }
+
+    @Override
+    public void setInnerValue(SettableByIndexData<?> udtValue, int position, Object value)
+    {
+        udtValue.setToNull(position);
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/Float.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/Float.java
new file mode 100644
index 0000000..99b4a4a
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/Float.java
@@ -0,0 +1,93 @@
+/*
+ * 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.cassandra.spark.data.types;
+
+import java.util.Comparator;
+
+import org.apache.cassandra.bridge.BigNumberConfig;
+import org.apache.cassandra.cql3.functions.types.SettableByIndexData;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.FloatType;
+import org.apache.cassandra.spark.data.NativeType;
+import org.apache.cassandra.spark.utils.RandomUtils;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+
+public class Float extends NativeType
+{
+    public static final Float INSTANCE = new Float();
+    private static final Comparator<java.lang.Float> FLOAT_COMPARATOR = java.lang.Float::compareTo;
+
+    @Override
+    public String name()
+    {
+        return "float";
+    }
+
+    @Override
+    public DataType sparkSqlType(BigNumberConfig bigNumberConfig)
+    {
+        return DataTypes.FloatType;
+    }
+
+    @Override
+    public AbstractType<?> dataType()
+    {
+        return FloatType.instance;
+    }
+
+    @Override
+    protected int compareTo(Object first, Object second)
+    {
+        return FLOAT_COMPARATOR.compare((java.lang.Float) first, (java.lang.Float) second);
+    }
+
+    @Override
+    protected Object nativeSparkSqlRowValue(GenericInternalRow row, int position)
+    {
+        return row.getFloat(position);
+    }
+
+    @Override
+    protected Object nativeSparkSqlRowValue(Row row, int position)
+    {
+        return row.getFloat(position);
+    }
+
+    @Override
+    public Object randomValue(int minCollectionSize)
+    {
+        return RandomUtils.RANDOM.nextFloat();
+    }
+
+    @Override
+    public void setInnerValue(SettableByIndexData<?> udtValue, int position, Object value)
+    {
+        udtValue.setFloat(position, (float) value);
+    }
+
+    @Override
+    public org.apache.cassandra.cql3.functions.types.DataType driverDataType(boolean isFrozen)
+    {
+        return org.apache.cassandra.cql3.functions.types.DataType.cfloat();
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/Inet.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/Inet.java
new file mode 100644
index 0000000..c684ca5
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/Inet.java
@@ -0,0 +1,86 @@
+/*
+ * 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.cassandra.spark.data.types;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+
+import com.google.common.net.InetAddresses;
+
+import org.apache.cassandra.cql3.functions.types.DataType;
+import org.apache.cassandra.cql3.functions.types.SettableByIndexData;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.InetAddressType;
+import org.apache.cassandra.spark.utils.RandomUtils;
+
+public class Inet extends BinaryBased
+{
+    public static final Inet INSTANCE = new Inet();
+
+    @Override
+    public String name()
+    {
+        return "inet";
+    }
+
+    @Override
+    public AbstractType<?> dataType()
+    {
+        return InetAddressType.instance;
+    }
+
+    @Override
+    public Object toSparkSqlType(Object value, boolean isFrozen)
+    {
+        return ((InetAddress) value).getAddress();
+    }
+
+    @Override
+    public Object toTestRowType(Object value)
+    {
+        try
+        {
+            return InetAddress.getByAddress((byte[]) value);
+        }
+        catch (UnknownHostException exception)
+        {
+            throw new RuntimeException(exception);
+        }
+    }
+
+    @Override
+    @SuppressWarnings("UnstableApiUsage")
+    public Object randomValue(int minCollectionSize)
+    {
+        return InetAddresses.fromInteger(RandomUtils.RANDOM.nextInt());
+    }
+
+    @Override
+    public void setInnerValue(SettableByIndexData<?> udtValue, int position, Object value)
+    {
+        udtValue.setInet(position, (InetAddress) value);
+    }
+
+    @Override
+    public DataType driverDataType(boolean isFrozen)
+    {
+        return DataType.inet();
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/Int.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/Int.java
new file mode 100644
index 0000000..bd6103e
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/Int.java
@@ -0,0 +1,93 @@
+/*
+ * 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.cassandra.spark.data.types;
+
+import java.util.Comparator;
+
+import org.apache.cassandra.bridge.BigNumberConfig;
+import org.apache.cassandra.cql3.functions.types.SettableByIndexData;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.spark.data.NativeType;
+import org.apache.cassandra.spark.utils.RandomUtils;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+
+public class Int extends NativeType
+{
+    public static final Int INSTANCE = new Int();
+    static final Comparator<Integer> INTEGER_COMPARATOR = Integer::compareTo;
+
+    @Override
+    public String name()
+    {
+        return "int";
+    }
+
+    @Override
+    public DataType sparkSqlType(BigNumberConfig bigNumberConfig)
+    {
+        return DataTypes.IntegerType;
+    }
+
+    @Override
+    public AbstractType<?> dataType()
+    {
+        return Int32Type.instance;
+    }
+
+    @Override
+    protected int compareTo(Object first, Object second)
+    {
+        return INTEGER_COMPARATOR.compare((Integer) first, (Integer) second);
+    }
+
+    @Override
+    protected Object nativeSparkSqlRowValue(GenericInternalRow row, int position)
+    {
+        return row.getInt(position);
+    }
+
+    @Override
+    protected Object nativeSparkSqlRowValue(Row row, int position)
+    {
+        return row.getInt(position);
+    }
+
+    @Override
+    public Object randomValue(int minCollectionSize)
+    {
+        return RandomUtils.RANDOM.nextInt();
+    }
+
+    @Override
+    public void setInnerValue(SettableByIndexData<?> udtValue, int position, Object value)
+    {
+        udtValue.setInt(position, (int) value);
+    }
+
+    @Override
+    public org.apache.cassandra.cql3.functions.types.DataType driverDataType(boolean isFrozen)
+    {
+        return org.apache.cassandra.cql3.functions.types.DataType.cint();
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/LongBased.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/LongBased.java
new file mode 100644
index 0000000..cf62403
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/LongBased.java
@@ -0,0 +1,65 @@
+/*
+ * 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.cassandra.spark.data.types;
+
+import java.util.Comparator;
+
+import org.apache.cassandra.bridge.BigNumberConfig;
+import org.apache.cassandra.spark.data.NativeType;
+import org.apache.cassandra.spark.utils.RandomUtils;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+
+public abstract class LongBased extends NativeType
+{
+    static final Comparator<Long> LONG_COMPARATOR = Long::compareTo;
+
+    @Override
+    public DataType sparkSqlType(BigNumberConfig bigNumberConfig)
+    {
+        return DataTypes.LongType;
+    }
+
+    @Override
+    protected int compareTo(Object first, Object second)
+    {
+        return LONG_COMPARATOR.compare((Long) first, (Long) second);
+    }
+
+    @Override
+    public Object randomValue(int minCollectionSize)
+    {
+        return (long) RandomUtils.randomPositiveInt(5_000_000);  // Keep within bound to avoid overflows
+    }
+
+    @Override
+    protected Object nativeSparkSqlRowValue(GenericInternalRow row, int position)
+    {
+        return row.getLong(position);
+    }
+
+    @Override
+    protected Object nativeSparkSqlRowValue(Row row, int position)
+    {
+        return row.getLong(position);
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/SmallInt.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/SmallInt.java
new file mode 100644
index 0000000..be89d85
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/SmallInt.java
@@ -0,0 +1,93 @@
+/*
+ * 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.cassandra.spark.data.types;
+
+import java.util.Comparator;
+
+import org.apache.cassandra.bridge.BigNumberConfig;
+import org.apache.cassandra.cql3.functions.types.SettableByIndexData;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.ShortType;
+import org.apache.cassandra.spark.data.NativeType;
+import org.apache.cassandra.spark.utils.RandomUtils;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+
+public class SmallInt extends NativeType
+{
+    public static final SmallInt INSTANCE = new SmallInt();
+    private static final Comparator<Short> SHORT_COMPARATOR = Short::compare;
+
+    @Override
+    public String name()
+    {
+        return "smallint";
+    }
+
+    @Override
+    public DataType sparkSqlType(BigNumberConfig bigNumberConfig)
+    {
+        return DataTypes.ShortType;
+    }
+
+    @Override
+    public AbstractType<?> dataType()
+    {
+        return ShortType.instance;
+    }
+
+    @Override
+    protected int compareTo(Object first, Object second)
+    {
+        return SHORT_COMPARATOR.compare((Short) first, (Short) second);
+    }
+
+    @Override
+    protected Object nativeSparkSqlRowValue(GenericInternalRow row, int position)
+    {
+        return row.getShort(position);
+    }
+
+    @Override
+    protected Object nativeSparkSqlRowValue(Row row, int position)
+    {
+        return row.getShort(position);
+    }
+
+    @Override
+    public Object randomValue(int minCollectionSize)
+    {
+        return (short) RandomUtils.RANDOM.nextInt(Short.MAX_VALUE + 1);
+    }
+
+    @Override
+    public void setInnerValue(SettableByIndexData<?> udtValue, int position, Object value)
+    {
+        udtValue.setShort(position, (short) value);
+    }
+
+    @Override
+    public org.apache.cassandra.cql3.functions.types.DataType driverDataType(boolean isFrozen)
+    {
+        return org.apache.cassandra.cql3.functions.types.DataType.smallint();
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/StringBased.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/StringBased.java
new file mode 100644
index 0000000..10d5e1e
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/StringBased.java
@@ -0,0 +1,102 @@
+/*
+ * 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.cassandra.spark.data.types;
+
+import java.util.Comparator;
+
+import org.apache.commons.lang3.RandomStringUtils;
+
+import org.apache.cassandra.bridge.BigNumberConfig;
+import org.apache.cassandra.cql3.functions.types.SettableByIndexData;
+import org.apache.cassandra.spark.data.NativeType;
+import org.apache.cassandra.spark.utils.RandomUtils;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.unsafe.types.UTF8String;
+
+public abstract class StringBased extends NativeType
+{
+    private static final Comparator<String> STRING_COMPARATOR = String::compareTo;
+
+    @Override
+    public Object toSparkSqlType(Object value, boolean isFrozen)
+    {
+        if (value == null)
+        {
+            return null;
+        }
+        return UTF8String.fromString(value.toString());  // UTF8String
+    }
+
+    @Override
+    protected int compareTo(Object first, Object second)
+    {
+        return STRING_COMPARATOR.compare(first.toString(), second.toString());
+    }
+
+    @Override
+    protected boolean equalsTo(Object first, Object second)
+    {
+        // UUID comparator is particularly slow because of UUID.fromString so compare for equality as strings
+        return first.equals(second);
+    }
+
+    @Override
+    public DataType sparkSqlType(BigNumberConfig bigNumberConfig)
+    {
+        return DataTypes.StringType;
+    }
+
+    @Override
+    public Object toTestRowType(Object value)
+    {
+        if (value instanceof UTF8String)
+        {
+            return ((UTF8String) value).toString();
+        }
+        return value;
+    }
+
+    @Override
+    protected Object nativeSparkSqlRowValue(GenericInternalRow row, int position)
+    {
+        return row.getString(position);
+    }
+
+    @Override
+    protected Object nativeSparkSqlRowValue(Row row, int position)
+    {
+        return row.getString(position);
+    }
+
+    @Override
+    public Object randomValue(int minCollectionSize)
+    {
+        return RandomStringUtils.randomAlphanumeric(RandomUtils.randomPositiveInt(32));
+    }
+
+    @Override
+    public void setInnerValue(SettableByIndexData<?> udtValue, int position, Object value)
+    {
+        udtValue.setString(position, (String) value);
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/Text.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/Text.java
new file mode 100644
index 0000000..e56fdec
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/Text.java
@@ -0,0 +1,47 @@
+/*
+ * 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.cassandra.spark.data.types;
+
+import org.apache.cassandra.cql3.functions.types.DataType;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+
+public class Text extends StringBased
+{
+    public static final Text INSTANCE = new Text();
+
+    @Override
+    public String name()
+    {
+        return "text";
+    }
+
+    @Override
+    public AbstractType<?> dataType()
+    {
+        return UTF8Type.instance;
+    }
+
+    @Override
+    public DataType driverDataType(boolean isFrozen)
+    {
+        return DataType.text();
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/Time.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/Time.java
new file mode 100644
index 0000000..39ce259
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/Time.java
@@ -0,0 +1,54 @@
+/*
+ * 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.cassandra.spark.data.types;
+
+import org.apache.cassandra.cql3.functions.types.DataType;
+import org.apache.cassandra.cql3.functions.types.SettableByIndexData;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.TimeType;
+
+public class Time extends LongBased
+{
+    public static final Time INSTANCE = new Time();
+
+    @Override
+    public String name()
+    {
+        return "time";
+    }
+
+    @Override
+    public AbstractType<?> dataType()
+    {
+        return TimeType.instance;
+    }
+
+    @Override
+    public void setInnerValue(SettableByIndexData<?> udtValue, int position, Object value)
+    {
+        udtValue.setTime(position, (long) value);
+    }
+
+    @Override
+    public DataType driverDataType(boolean isFrozen)
+    {
+        return DataType.time();
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/TimeUUID.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/TimeUUID.java
new file mode 100644
index 0000000..a5d2431
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/TimeUUID.java
@@ -0,0 +1,54 @@
+/*
+ * 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.cassandra.spark.data.types;
+
+import org.apache.cassandra.cql3.functions.types.DataType;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.TimeUUIDType;
+import org.apache.cassandra.utils.UUIDGen;
+
+public class TimeUUID extends UUID
+{
+    public static final TimeUUID INSTANCE = new TimeUUID();
+
+    @Override
+    public String name()
+    {
+        return "timeuuid";
+    }
+
+    @Override
+    public AbstractType<?> dataType()
+    {
+        return TimeUUIDType.instance;
+    }
+
+    @Override
+    public Object randomValue(int minCollectionSize)
+    {
+        return UUIDGen.getTimeUUID();
+    }
+
+    @Override
+    public DataType driverDataType(boolean isFrozen)
+    {
+        return DataType.timeuuid();
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/Timestamp.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/Timestamp.java
new file mode 100644
index 0000000..51dcd6a
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/Timestamp.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.cassandra.spark.data.types;
+
+import org.apache.cassandra.bridge.BigNumberConfig;
+import org.apache.cassandra.cql3.functions.types.SettableByIndexData;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.TimestampType;
+import org.apache.cassandra.spark.data.NativeType;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+
+public class Timestamp extends NativeType
+{
+    public static final Timestamp INSTANCE = new Timestamp();
+
+    @Override
+    public String name()
+    {
+        return "timestamp";
+    }
+
+    @Override
+    public DataType sparkSqlType(BigNumberConfig bigNumberConfig)
+    {
+        return DataTypes.TimestampType;
+    }
+
+    @Override
+    public AbstractType<?> dataType()
+    {
+        return TimestampType.instance;
+    }
+
+    @Override
+    public Object toSparkSqlType(Object value, boolean isFrozen)
+    {
+        return ((java.util.Date) value).getTime() * 1000L;  // long
+    }
+
+    @Override
+    protected int compareTo(Object first, Object second)
+    {
+        return BigInt.LONG_COMPARATOR.compare((Long) first, (Long) second);
+    }
+
+    @Override
+    protected Object nativeSparkSqlRowValue(GenericInternalRow row, int position)
+    {
+        return row.getLong(position);
+    }
+
+    @Override
+    protected Object nativeSparkSqlRowValue(Row row, int position)
+    {
+        return new java.util.Date(row.getTimestamp(position).getTime());
+    }
+
+    @Override
+    public Object toTestRowType(Object value)
+    {
+        if (value instanceof java.util.Date)
+        {
+            return value;
+        }
+        return new java.util.Date((long) value / 1000L);
+    }
+
+    @Override
+    public void setInnerValue(SettableByIndexData<?> udtValue, int position, Object value)
+    {
+        udtValue.setTimestamp(position, (java.util.Date) value);
+    }
+
+    @Override
+    public Object randomValue(int minCollectionSize)
+    {
+        return new java.util.Date();
+    }
+
+    @Override
+    public org.apache.cassandra.cql3.functions.types.DataType driverDataType(boolean isFrozen)
+    {
+        return org.apache.cassandra.cql3.functions.types.DataType.timestamp();
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/TinyInt.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/TinyInt.java
new file mode 100644
index 0000000..b00cf72
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/TinyInt.java
@@ -0,0 +1,98 @@
+/*
+ * 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.cassandra.spark.data.types;
+
+import java.util.Comparator;
+
+import org.apache.cassandra.bridge.BigNumberConfig;
+import org.apache.cassandra.cql3.functions.types.SettableByIndexData;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.ByteType;
+import org.apache.cassandra.spark.data.NativeType;
+import org.apache.cassandra.spark.utils.RandomUtils;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+
+public class TinyInt extends NativeType
+{
+    public static final TinyInt INSTANCE = new TinyInt();
+    private static final Comparator<Byte> BYTE_COMPARATOR = TinyInt::compareBytes;
+
+    private static int compareBytes(byte first, byte second)
+    {
+        return first - second;  // Safe because of the range being restricted
+    }
+
+    @Override
+    public String name()
+    {
+        return "tinyint";
+    }
+
+    @Override
+    public DataType sparkSqlType(BigNumberConfig bigNumberConfig)
+    {
+        return DataTypes.ByteType;
+    }
+
+    @Override
+    public AbstractType<?> dataType()
+    {
+        return ByteType.instance;
+    }
+
+    @Override
+    protected int compareTo(Object first, Object second)
+    {
+        return BYTE_COMPARATOR.compare((Byte) first, (Byte) second);
+    }
+
+    @Override
+    protected Object nativeSparkSqlRowValue(GenericInternalRow row, int position)
+    {
+        return row.getByte(position);
+    }
+
+    @Override
+    protected Object nativeSparkSqlRowValue(Row row, int position)
+    {
+        return row.getByte(position);
+    }
+
+    @Override
+    public Object randomValue(int minCollectionSize)
+    {
+        return RandomUtils.randomBytes(1)[0];
+    }
+
+    @Override
+    public void setInnerValue(SettableByIndexData<?> udtValue, int position, Object value)
+    {
+        udtValue.setByte(position, (byte) value);
+    }
+
+    @Override
+    public org.apache.cassandra.cql3.functions.types.DataType driverDataType(boolean isFrozen)
+    {
+        return org.apache.cassandra.cql3.functions.types.DataType.tinyint();
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/UUID.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/UUID.java
new file mode 100644
index 0000000..54f7161
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/UUID.java
@@ -0,0 +1,75 @@
+/*
+ * 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.cassandra.spark.data.types;
+
+import java.util.Comparator;
+
+import org.apache.cassandra.cql3.functions.types.DataType;
+import org.apache.cassandra.cql3.functions.types.SettableByIndexData;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.UUIDType;
+
+public class UUID extends StringBased
+{
+    public static final UUID INSTANCE = new UUID();
+    private static final Comparator<String> UUID_COMPARATOR = Comparator.comparing(java.util.UUID::fromString);
+
+    @Override
+    public String name()
+    {
+        return "uuid";
+    }
+
+    @Override
+    public AbstractType<?> dataType()
+    {
+        return UUIDType.instance;
+    }
+
+    @Override
+    protected int compareTo(Object first, Object second)
+    {
+        return UUID_COMPARATOR.compare(first.toString(), second.toString());
+    }
+
+    @Override
+    public Object toTestRowType(Object value)
+    {
+        return java.util.UUID.fromString(value.toString());
+    }
+
+    @Override
+    public DataType driverDataType(boolean isFrozen)
+    {
+        return DataType.uuid();
+    }
+
+    @Override
+    public void setInnerValue(SettableByIndexData<?> udtValue, int position, Object value)
+    {
+        udtValue.setUUID(position, (java.util.UUID) value);
+    }
+
+    @Override
+    public Object randomValue(int minCollectionSize)
+    {
+        return java.util.UUID.randomUUID();
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/VarChar.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/VarChar.java
new file mode 100644
index 0000000..6c23406
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/VarChar.java
@@ -0,0 +1,47 @@
+/*
+ * 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.cassandra.spark.data.types;
+
+import org.apache.cassandra.cql3.functions.types.DataType;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+
+public class VarChar extends StringBased
+{
+    public static final VarChar INSTANCE = new VarChar();
+
+    @Override
+    public String name()
+    {
+        return "varchar";
+    }
+
+    @Override
+    public AbstractType<?> dataType()
+    {
+        return UTF8Type.instance;
+    }
+
+    @Override
+    public DataType driverDataType(boolean isFrozen)
+    {
+        return DataType.varchar();
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/VarInt.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/VarInt.java
new file mode 100644
index 0000000..6635920
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/data/types/VarInt.java
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.cassandra.spark.data.types;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+
+import org.apache.cassandra.bridge.BigNumberConfig;
+import org.apache.cassandra.cql3.functions.types.SettableByIndexData;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.IntegerType;
+import org.apache.cassandra.spark.utils.RandomUtils;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+
+public class VarInt extends Decimal
+{
+    public static final VarInt INSTANCE = new VarInt();
+
+    @Override
+    public String name()
+    {
+        return "varint";
+    }
+
+    @Override
+    public DataType sparkSqlType(BigNumberConfig bigNumberConfig)
+    {
+        return DataTypes.createDecimalType(bigNumberConfig.bigIntegerPrecision(), bigNumberConfig.bigIntegerScale());
+    }
+
+    @Override
+    public AbstractType<?> dataType()
+    {
+        return IntegerType.instance;
+    }
+
+    @Override
+    public Object toTestRowType(Object value)
+    {
+        if (value instanceof BigInteger)
+        {
+            return value;
+        }
+        else if (value instanceof BigDecimal)
+        {
+            return ((BigDecimal) value).toBigInteger();
+        }
+        return ((org.apache.spark.sql.types.Decimal) value).toJavaBigInteger();
+    }
+
+    @Override
+    public Object toSparkSqlType(Object value, boolean isFrozen)
+    {
+        return org.apache.spark.sql.types.Decimal.apply((BigInteger) value);
+    }
+
+    @Override
+    protected Object nativeSparkSqlRowValue(GenericInternalRow row, int position)
+    {
+        return row.getDecimal(position, BigNumberConfig.DEFAULT.bigIntegerPrecision(), BigNumberConfig.DEFAULT.bigIntegerScale());
+    }
+
+    @Override
+    protected Object nativeSparkSqlRowValue(Row row, int position)
+    {
+        return row.getDecimal(position).toBigInteger();
+    }
+
+    @Override
+    public Object randomValue(int minCollectionSize)
+    {
+        return new BigInteger(BigNumberConfig.DEFAULT.bigIntegerPrecision(), RandomUtils.RANDOM);
+    }
+
+    @Override
+    public void setInnerValue(SettableByIndexData<?> udtValue, int position, Object value)
+    {
+        udtValue.setVarint(position, (BigInteger) value);
+    }
+
+    @Override
+    public org.apache.cassandra.cql3.functions.types.DataType driverDataType(boolean isFrozen)
+    {
+        return org.apache.cassandra.cql3.functions.types.DataType.varint();
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/reader/AbstractStreamScanner.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/reader/AbstractStreamScanner.java
new file mode 100644
index 0000000..9cae19f
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/reader/AbstractStreamScanner.java
@@ -0,0 +1,606 @@
+/*
+ * 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.cassandra.spark.reader;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.cassandra.db.Clustering;
+import org.apache.cassandra.db.ClusteringPrefix;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.ListType;
+import org.apache.cassandra.db.marshal.MapType;
+import org.apache.cassandra.db.marshal.SetType;
+import org.apache.cassandra.db.marshal.UserType;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
+import org.apache.cassandra.db.rows.Cell;
+import org.apache.cassandra.db.rows.ColumnData;
+import org.apache.cassandra.db.rows.ComplexColumnData;
+import org.apache.cassandra.db.rows.RangeTombstoneMarker;
+import org.apache.cassandra.db.rows.Row;
+import org.apache.cassandra.db.rows.Unfiltered;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.dht.RandomPartitioner;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.spark.data.partitioner.Partitioner;
+import org.apache.cassandra.spark.reader.common.SSTableStreamException;
+import org.apache.cassandra.spark.utils.TimeProvider;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.jetbrains.annotations.NotNull;
+
+public abstract class AbstractStreamScanner implements StreamScanner, Closeable
+{
+    // All partitions in the SSTable
+    private UnfilteredPartitionIterator allPartitions;
+    // A single partition, containing rows and/or range tombstones
+    private UnfilteredRowIterator partition;
+    // The static row of the current partition, which may be empty
+    @SuppressWarnings("FieldCanBeLocal")
+    private Row staticRow;
+    // Current atom (row or range tombstone) being processed
+    private Unfiltered unfiltered;
+    // If processing a row this holds the state of iterating that row
+    private Iterator<ColumnData> columns;
+    // State of processing data for a single column in a row (which may be multi-celled in the case of complex columns)
+    protected ColumnDataState columnData;
+
+    @NotNull
+    final TableMetadata metadata;
+
+    @NotNull
+    protected final TimeProvider timeProvider;
+
+    protected final Rid rid = new Rid();
+
+    AbstractStreamScanner(@NotNull TableMetadata metadata,
+                          @NotNull Partitioner partitionerType,
+                          @NotNull TimeProvider timeProvider)
+    {
+        this.metadata = metadata.unbuild()
+                                .partitioner(partitionerType == Partitioner.Murmur3Partitioner
+                                        ? new Murmur3Partitioner()
+                                        : new RandomPartitioner())
+                                .build();
+        this.timeProvider = timeProvider;
+
+        // Counter tables are not supported
+        if (metadata.isCounter())
+        {
+            throw new IllegalArgumentException(
+                    String.format("Streaming reads of SSTables from counter tables are not supported, "
+                                + "rejecting stream of data from %s.%s",
+                                  metadata.keyspace, metadata.name));
+        }
+    }
+
+    @Override
+    public Rid rid()
+    {
+        return rid;
+    }
+
+    /* Abstract methods */
+
+    abstract UnfilteredPartitionIterator initializePartitions();
+
+    @Override
+    public abstract void close() throws IOException;
+
+    protected abstract void handleRowTombstone(Row row);
+
+    protected abstract void handlePartitionTombstone(UnfilteredRowIterator partition);
+
+    protected abstract void handleCellTombstone();
+
+    protected abstract void handleCellTombstoneInComplex(Cell<?> cell);
+
+    protected abstract void handleRangeTombstone(RangeTombstoneMarker marker);
+
+    @Override
+    public void advanceToNextColumn()
+    {
+        columnData.consume();
+    }
+
+    // CHECKSTYLE IGNORE: Long method
+    @Override
+    public boolean hasNext() throws IOException
+    {
+        if (allPartitions == null)
+        {
+            allPartitions = initializePartitions();
+        }
+
+        while (true)
+        {
+            if (partition == null)
+            {
+                try
+                {
+                    // We've exhausted the partition iterator
+                    if (allPartitions.hasNext())
+                    {
+                        // Advance to next partition
+                        partition = allPartitions.next();
+
+                        if (partition.partitionLevelDeletion().isLive())
+                        {
+                            // Reset rid with new partition key
+                            rid.setPartitionKeyCopy(partition.partitionKey().getKey(),
+                                                    ReaderUtils.tokenToBigInteger(partition.partitionKey().getToken()));
+                        }
+                        else
+                        {
+                            // There's a partition-level delete
+                            handlePartitionTombstone(partition);
+                            return true;
+                        }
+                    }
+                    else
+                    {
+                        return false;
+                    }
+                }
+                catch (SSTableStreamException exception)
+                {
+                    throw exception.getIOException();
+                }
+
+                // If the partition has a non-empty static row, grab its columns,
+                // so we process those before moving onto its atoms (the Unfiltered instances)
+                staticRow = partition.staticRow();
+                if (!staticRow.isEmpty())
+                {
+                    columns = staticRow.iterator();
+                    prepareColumnData();
+                    return true;
+                }
+            }
+
+            // We may be in the midst of processing some multi-cell column data,
+            // if so, we'll resume that where we left off
+            if (columnData != null && columnData.hasData())
+            {
+                return true;
+            }
+
+            // Continue to process columns of the last read row, which may be static
+            if (columns != null && columns.hasNext())
+            {
+                prepareColumnData();
+                return true;
+            }
+
+            // Current row was exhausted (or none were present), so move to the next atom
+            columns = null;
+            try
+            {
+                // Advance to next unfiltered
+                rid.setIsUpdate(false);  // Reset isUpdate flag
+                if (partition.hasNext())
+                {
+                    unfiltered = partition.next();
+                }
+                else
+                {
+                    // Current partition is exhausted
+                    partition = null;
+                    unfiltered = null;
+
+                    // Produce a spark row if there are range tombstone markers
+                    if (rid.hasRangeTombstoneMarkers())
+                    {
+                        // The current partition is exhusted and ready to produce a spark row for the range tombstones
+                        rid.setShouldConsumeRangeTombstoneMarkers(true);
+                        return true;
+                    }
+                }
+            }
+            catch (SSTableStreamException exception)
+            {
+                throw exception.getIOException();
+            }
+
+            if (unfiltered != null)
+            {
+                if (unfiltered.isRow())
+                {
+                    Row row = (Row) unfiltered;
+
+                    // There is a CQL row level delete
+                    if (!row.deletion().isLive())
+                    {
+                        handleRowTombstone(row);
+                        return true;
+                    }
+
+                    // For non-compact tables, set up a ClusteringColumnDataState to emit a Rid that emulates a
+                    // pre-3.0 CQL row marker. This is necessary for backwards compatibility with 2.1 & 2.0 output,
+                    // and also for tables with only primary key columns defined.
+                    // An empty PKLI is the 3.0 equivalent of having no row marker (e.g. row modifications via
+                    // UPDATE not INSERT) so we don't emit a fake row marker in that case.
+                    boolean emptyLiveness = row.primaryKeyLivenessInfo().isEmpty();
+                    rid.setIsUpdate(emptyLiveness);
+                    if (!emptyLiveness)
+                    {
+                        if (TableMetadata.Flag.isCQLTable(metadata.flags))
+                        {
+                            columnData = new ClusteringColumnDataState(row.clustering());
+                        }
+                        columns = row.iterator();
+                        return true;
+                    }
+
+                    // The row's actual columns may be empty, in which case we'll simply skip over them during the next
+                    // iteration and move to the next unfiltered. So then only the row marker and/or row deletion (if
+                    // either are present) will get emitted
+                    columns = row.iterator();
+                }
+                else if (unfiltered.isRangeTombstoneMarker())
+                {
+                    // Range tombstone can get complicated:
+                    // - In the most simple case, that is a DELETE statement with a single clustering key range, we
+                    //   expect the UnfilteredRowIterator with 2 markers, i.e. open and close range tombstone markers
+                    // - In a slightly more complicated case, it contains IN operator (on prior clustering keys), we
+                    //   expect the UnfilteredRowIterator with 2 * N markers, where N is the number of values specified
+                    //   for IN
+                    // - In the most complicated case, client could comopse a complex partition update with a BATCH
+                    //   statement; it could have those further scenarios: (only discussing the statements applying to
+                    //   the same partition key)
+                    //   - Multiple disjoint ranges => we should expect 2 * N markers, where N is the number of ranges
+                    //   - Overlapping ranges with the same timestamp => we should expect 2 markers, considering the
+                    //     overlapping ranges are merged into a single one. (as the boundary is omitted)
+                    //   - Overlapping ranges with different timestamp ==> we should expect 3 markers, i.e. open bound,
+                    //     boundary and end bound
+                    //   - Ranges mixed with INSERT! => The order of the unfiltered (i.e. Row/RangeTombstoneMarker) is
+                    //     determined by comparing the row clustering with the bounds of the ranges.
+                    //     See o.a.c.d.r.RowAndDeletionMergeIterator
+                    RangeTombstoneMarker rangeTombstoneMarker = (RangeTombstoneMarker) unfiltered;
+                    // We encode the ranges within the same spark row. Therefore, it needs to keep the markers when
+                    // iterating through the partition, and _only_ generate a spark row with range tombstone info when
+                    // exhausting the partition / UnfilteredRowIterator.
+                    handleRangeTombstone(rangeTombstoneMarker);
+                    // Continue to consume the next unfiltered row/marker
+                }
+                else
+                {
+                    // As of Cassandra 4, the unfiltered kind can either be row or range tombstone marker,
+                    // see o.a.c.db.rows.Unfiltered.Kind; having the else branch only for completeness
+                    throw new IllegalStateException("Encountered unknown Unfiltered kind");
+                }
+            }
+        }
+    }
+
+    /**
+     * Prepare the columnData to be consumed the next
+     */
+    private void prepareColumnData()
+    {
+        ColumnData data = columns.next();
+        if (data.column().isComplex())
+        {
+            columnData = new ComplexDataState(data.column().isStatic() ? Clustering.STATIC_CLUSTERING
+                                                                       : unfiltered.clustering(),
+                                              (ComplexColumnData) data);
+        }
+        else
+        {
+            columnData = new SimpleColumnDataState(data.column().isStatic() ? Clustering.STATIC_CLUSTERING
+                                                                            : unfiltered.clustering(),
+                                                   data);
+        }
+    }
+
+    private interface ColumnDataState
+    {
+        /**
+         * Indicate whether the column has data
+         *
+         * @return true if it has data to be consumed
+         */
+        boolean hasData();
+
+        /**
+         * Consume the data in the column
+         */
+        void consume();
+    }
+
+    /**
+     * Maps clustering values to column data, to emulate CQL row markers which were removed in Cassandra 3.0,
+     * but which we must still emit Rid for in order to preserve backwards compatibility
+     * and to handle tables containing only primary key columns
+     */
+    protected final class ClusteringColumnDataState implements ColumnDataState
+    {
+        private boolean consumed = false;
+        private final ClusteringPrefix clustering;
+
+        ClusteringColumnDataState(ClusteringPrefix clustering)
+        {
+            this.clustering = clustering;
+        }
+
+        @Override
+        public boolean hasData()
+        {
+            return !consumed;
+        }
+
+        @Override
+        public void consume()
+        {
+            if (!consumed)
+            {
+                rid.setColumnNameCopy(ReaderUtils.encodeCellName(metadata,
+                                                                 clustering,
+                                                                 ByteBufferUtil.EMPTY_BYTE_BUFFER,
+                                                                 null));
+                rid.setValueCopy(ByteBufferUtil.EMPTY_BYTE_BUFFER);
+                consumed = true;
+            }
+            else
+            {
+                throw new UnsupportedOperationException();
+            }
+        }
+    }
+
+    /**
+     * Holds current processing state of any simple column data
+     */
+    private final class SimpleColumnDataState implements ColumnDataState
+    {
+        private ClusteringPrefix clustering;
+        private final Cell cell;
+
+        private SimpleColumnDataState(ClusteringPrefix clustering, ColumnData data)
+        {
+            Preconditions.checkArgument(data.column().isSimple(), "The type of the ColumnData should be simple");
+            this.clustering = clustering;
+            this.cell = (Cell) data;
+        }
+
+        @Override
+        public boolean hasData()
+        {
+            return (clustering != null);
+        }
+
+        @Override
+        public void consume()
+        {
+            boolean isStatic = cell.column().isStatic();
+            rid.setColumnNameCopy(ReaderUtils.encodeCellName(metadata,
+                                                             isStatic ? Clustering.STATIC_CLUSTERING : clustering,
+                                                             cell.column().name.bytes,
+                                                             null));
+            if (cell.isTombstone())
+            {
+                handleCellTombstone();
+            }
+            else
+            {
+                rid.setValueCopy(cell.buffer());
+            }
+            rid.setTimestamp(cell.timestamp());
+            // Null out clustering so hasData will return false
+            clustering = null;
+        }
+    }
+
+    /**
+     * Holds current processing state of any complex column data
+     */
+    private final class ComplexDataState implements ColumnDataState
+    {
+        private final ColumnMetadata column;
+        private ClusteringPrefix clustering;
+        private final Iterator<Cell<?>> cells;
+        private final int cellCount;
+        private final DeletionTime deletionTime;
+
+        private ComplexDataState(ClusteringPrefix clustering, ComplexColumnData data)
+        {
+            this.clustering = clustering;
+            this.column = data.column();
+            this.cells = data.iterator();
+            this.cellCount = data.cellsCount();
+            this.deletionTime = data.complexDeletion();
+        }
+
+        @Override
+        public boolean hasData()
+        {
+            return clustering != null && cells.hasNext();
+        }
+
+        @Override
+        public void consume()
+        {
+            rid.setColumnNameCopy(ReaderUtils.encodeCellName(metadata,
+                                                             clustering,
+                                                             column.name.bytes,
+                                                             ByteBufferUtil.EMPTY_BYTE_BUFFER));
+            // The complex data is live, but there could be element deletion inside; check for it later in the block
+            if (deletionTime.isLive())
+            {
+                ComplexTypeBuffer buffer = ComplexTypeBuffer.newBuffer(column.type, cellCount);
+                long maxTimestamp = Long.MIN_VALUE;
+                while (cells.hasNext())
+                {
+                    Cell<?> cell = cells.next();
+                    // Re: isLive vs. isTombstone - isLive considers TTL so that if a cell is expiring soon,
+                    // it is handled as tombstone
+                    if (cell.isLive(timeProvider.nowInTruncatedSeconds()))
+                    {
+                        buffer.addCell(cell);
+                    }
+                    else
+                    {
+                        // Only adds the tombstoned cell when running as a CDC job
+                        handleCellTombstoneInComplex(cell);
+                    }
+                    // In the case the cell is deleted, the deletion time is also the cell's timestamp
+                    maxTimestamp = Math.max(maxTimestamp, cell.timestamp());
+                }
+                // In the case of CDC, consuming the mutation contains cell tombstones
+                // results into an empty buffer built
+                if (rid.hasCellTombstoneInComplex())
+                {
+                    rid.setValueCopy(null);
+                }
+                else
+                {
+                    rid.setValueCopy(buffer.build());
+                }
+                rid.setTimestamp(maxTimestamp);
+            }
+            else
+            {
+                // The entire collection/UDT is deleted
+                handleCellTombstone();
+                rid.setTimestamp(deletionTime.markedForDeleteAt());
+            }
+
+            // Null out clustering to indicate no data
+            clustering = null;
+        }
+    }
+
+    private abstract static class ComplexTypeBuffer
+    {
+        private final List<ByteBuffer> buffers;
+        private final int cellCount;
+        private int length = 0;
+
+        ComplexTypeBuffer(int cellCount, int bufferSize)
+        {
+            this.cellCount = cellCount;
+            this.buffers = new ArrayList<>(bufferSize);
+        }
+
+        static ComplexTypeBuffer newBuffer(AbstractType<?> type, int cellCount)
+        {
+            ComplexTypeBuffer buffer;
+            if (type instanceof SetType)
+            {
+                buffer = new SetBuffer(cellCount);
+            }
+            else if (type instanceof ListType)
+            {
+                buffer = new ListBuffer(cellCount);
+            }
+            else if (type instanceof MapType)
+            {
+                buffer = new MapBuffer(cellCount);
+            }
+            else if (type instanceof UserType)
+            {
+                buffer = new UdtBuffer(cellCount);
+            }
+            else
+            {
+                throw new IllegalStateException("Unexpected type deserializing CQL Collection: " + type);
+            }
+            return buffer;
+        }
+
+        void addCell(Cell cell)
+        {
+            add(cell.buffer());  // Copy over value
+        }
+
+        void add(ByteBuffer buffer)
+        {
+            buffers.add(buffer);
+            length += buffer.remaining();
+        }
+
+        ByteBuffer build()
+        {
+            ByteBuffer result = ByteBuffer.allocate(4 + (buffers.size() * 4) + length);
+            result.putInt(cellCount);
+            for (ByteBuffer buffer : buffers)
+            {
+                result.putInt(buffer.remaining());
+                result.put(buffer);
+            }
+            // Cast to ByteBuffer required when compiling with Java 8
+            return (ByteBuffer) result.flip();
+        }
+    }
+
+    private static class SetBuffer extends ComplexTypeBuffer
+    {
+        SetBuffer(int cellCount)
+        {
+            super(cellCount, cellCount);
+        }
+
+        @Override
+        void addCell(Cell cell)
+        {
+            add(cell.path().get(0));  // Set - copy over key
+        }
+    }
+
+    private static class ListBuffer extends ComplexTypeBuffer
+    {
+        ListBuffer(int cellCount)
+        {
+            super(cellCount, cellCount);
+        }
+    }
+
+    private static class MapBuffer extends ComplexTypeBuffer
+    {
+
+        MapBuffer(int cellCount)
+        {
+            super(cellCount, cellCount * 2);
+        }
+
+        @Override
+        void addCell(Cell cell)
+        {
+            add(cell.path().get(0));  // Map - copy over key and value
+            super.addCell(cell);
+        }
+    }
+
+    private static class UdtBuffer extends ComplexTypeBuffer
+    {
+        UdtBuffer(int cellCount)
+        {
+            super(cellCount, cellCount);
+        }
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/reader/CdcScannerBuilder.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/reader/CdcScannerBuilder.java
new file mode 100644
index 0000000..bd7d17f
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/reader/CdcScannerBuilder.java
@@ -0,0 +1,536 @@
+/*
+ * 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.cassandra.spark.reader;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.PriorityQueue;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.commitlog.BufferingCommitLogReader;
+import org.apache.cassandra.db.commitlog.PartitionUpdateWrapper;
+import org.apache.cassandra.db.marshal.ListType;
+import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
+import org.apache.cassandra.db.rows.Cell;
+import org.apache.cassandra.db.rows.CellPath;
+import org.apache.cassandra.db.rows.RangeTombstoneMarker;
+import org.apache.cassandra.db.rows.Row;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.spark.cdc.CommitLog;
+import org.apache.cassandra.spark.cdc.CommitLogProvider;
+import org.apache.cassandra.spark.cdc.watermarker.Watermarker;
+import org.apache.cassandra.spark.data.partitioner.CassandraInstance;
+import org.apache.cassandra.spark.data.partitioner.Partitioner;
+import org.apache.cassandra.spark.sparksql.RangeTombstoneMarkerImplementation;
+import org.apache.cassandra.spark.sparksql.filters.CdcOffsetFilter;
+import org.apache.cassandra.spark.sparksql.filters.SparkRangeFilter;
+import org.apache.cassandra.spark.stats.Stats;
+import org.apache.cassandra.spark.utils.FutureUtils;
+import org.apache.cassandra.spark.utils.TimeProvider;
+import org.apache.spark.TaskContext;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+public class CdcScannerBuilder
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(CdcScannerBuilder.class);
+
+    // Match both legacy and new version of CommitLogs, for example: CommitLog-12345.log and CommitLog-4-12345.log
+    private static final Pattern COMMIT_LOG_FILE_PATTERN = Pattern.compile("CommitLog(-\\d+)*-(\\d+).log");
+    private static final CompletableFuture<BufferingCommitLogReader.Result> NO_OP_FUTURE =
+            CompletableFuture.completedFuture(null);
+
+    final TableMetadata table;
+    final Partitioner partitioner;
+    final Stats stats;
+    final Map<CassandraInstance, CompletableFuture<List<PartitionUpdateWrapper>>> futures;
+    final int minimumReplicasPerMutation;
+    @Nullable
+    private final SparkRangeFilter sparkRangeFilter;
+    @Nullable
+    private final CdcOffsetFilter offsetFilter;
+    @NotNull
+    final Watermarker watermarker;
+    private final int partitionId;
+    private final long startTimeNanos;
+    @NotNull
+    private final TimeProvider timeProvider;
+
+    // CHECKSTYLE IGNORE: Constructor with many parameters
+    public CdcScannerBuilder(int partitionId,
+                             TableMetadata table,
+                             Partitioner partitioner,
+                             CommitLogProvider commitLogs,
+                             Stats stats,
+                             @Nullable SparkRangeFilter sparkRangeFilter,
+                             @Nullable CdcOffsetFilter offsetFilter,
+                             int minimumReplicasPerMutation,
+                             @NotNull Watermarker jobWatermarker,
+                             @NotNull String jobId,
+                             @NotNull ExecutorService executorService,
+                             @NotNull TimeProvider timeProvider)
+    {
+        this.table = table;
+        this.partitioner = partitioner;
+        this.stats = stats;
+        this.sparkRangeFilter = sparkRangeFilter;
+        this.offsetFilter = offsetFilter;
+        this.watermarker = jobWatermarker.instance(jobId);
+        Preconditions.checkArgument(minimumReplicasPerMutation >= 1,
+                                    "minimumReplicasPerMutation should be at least 1");
+        this.minimumReplicasPerMutation = minimumReplicasPerMutation;
+        this.startTimeNanos = System.nanoTime();
+        this.timeProvider = timeProvider;
+
+        Map<CassandraInstance, List<CommitLog>> logs = commitLogs
+                .logs()
+                .collect(Collectors.groupingBy(CommitLog::instance, Collectors.toList()));
+        Map<CassandraInstance, CommitLog.Marker> markers = logs.keySet().stream()
+                .map(watermarker::highWaterMark)
+                .filter(Objects::nonNull)
+                .collect(Collectors.toMap(CommitLog.Marker::instance, Function.identity()));
+
+        this.partitionId = partitionId;
+        LOGGER.info("Opening CdcScanner numInstances={} start={} maxAgeMicros={} partitionId={} listLogsTimeNanos={}",
+                    logs.size(),
+                    offsetFilter != null ? offsetFilter.start().getTimestampMicros() : null,
+                    offsetFilter != null ? offsetFilter.maxAgeMicros() : null,
+                    partitionId,
+                    System.nanoTime() - startTimeNanos);
+
+        this.futures = logs.entrySet().stream()
+                .collect(Collectors.toMap(Map.Entry::getKey, entry -> openInstanceAsync(entry.getValue(),
+                                                                                        markers.get(entry.getKey()),
+                                                                                        executorService)));
+    }
+
+    private static boolean skipCommitLog(@NotNull CommitLog log, @Nullable CommitLog.Marker highwaterMark)
+    {
+        if (highwaterMark == null)
+        {
+            return false;
+        }
+        Long segmentId = extractSegmentId(log);
+        if (segmentId != null)
+        {
+            // Only read CommitLog if greater than or equal to previously read CommitLog segmentId
+            return segmentId < highwaterMark.segmentId();
+        }
+        return true;
+    }
+
+    @Nullable
+    public static Long extractSegmentId(@NotNull CommitLog log)
+    {
+        return extractSegmentId(log.name());
+    }
+
+    @Nullable
+    public static Long extractSegmentId(@NotNull String filename)
+    {
+        Matcher matcher = CdcScannerBuilder.COMMIT_LOG_FILE_PATTERN.matcher(filename);
+        if (matcher.matches())
+        {
+            try
+            {
+                return Long.parseLong(matcher.group(2));
+            }
+            catch (NumberFormatException exception)
+            {
+                LOGGER.error("Could not parse commit log segmentId name={}", filename, exception);
+                return null;
+            }
+        }
+        LOGGER.error("Could not parse commit log filename name={}", filename);
+        return null;  // Cannot extract segment id
+    }
+
+    private CompletableFuture<List<PartitionUpdateWrapper>> openInstanceAsync(@NotNull List<CommitLog> logs,
+                                                                              @Nullable CommitLog.Marker highWaterMark,
+                                                                              @NotNull ExecutorService executorService)
+    {
+        // Read all CommitLogs on instance async and combine into single future,
+        // if we fail to read any CommitLog on the instance we fail this instance
+        List<CompletableFuture<BufferingCommitLogReader.Result>> futures = logs.stream()
+                .map(log -> openReaderAsync(log, highWaterMark, executorService))
+                .collect(Collectors.toList());
+        return FutureUtils.combine(futures)
+                          .thenApply(result -> {
+                              // Update highwater mark on success, if instance fails we don't update
+                              // highwater mark so resume from original position on next attempt
+                              result.stream()
+                                    .map(BufferingCommitLogReader.Result::marker)
+                                    .max(CommitLog.Marker::compareTo)
+                                    .ifPresent(watermarker::updateHighWaterMark);
+
+                              // Combine all updates into single list
+                              return result.stream()
+                                           .map(BufferingCommitLogReader.Result::updates)
+                                           .flatMap(Collection::stream)
+                                           .collect(Collectors.toList());
+                          });
+    }
+
+    private CompletableFuture<BufferingCommitLogReader.Result> openReaderAsync(@NotNull CommitLog log,
+                                                                               @Nullable CommitLog.Marker highWaterMark,
+                                                                               @NotNull ExecutorService executorService)
+    {
+        if (skipCommitLog(log, highWaterMark))
+        {
+            return NO_OP_FUTURE;
+        }
+        return CompletableFuture.supplyAsync(() -> openReader(log, highWaterMark), executorService);
+    }
+
+    @Nullable
+    private BufferingCommitLogReader.Result openReader(@NotNull CommitLog log, @Nullable CommitLog.Marker highWaterMark)
+    {
+        long startTimeNanos = System.nanoTime();
+        LOGGER.info("Opening BufferingCommitLogReader instance={} log={} high='{}' partitionId={}",
+                    log.instance().nodeName(), log.name(), highWaterMark, partitionId);
+        try (BufferingCommitLogReader reader =
+                new BufferingCommitLogReader(table, offsetFilter, log, sparkRangeFilter, highWaterMark, partitionId))
+        {
+            if (reader.isReadable())
+            {
+                return reader.result();
+            }
+        }
+        finally
+        {
+            LOGGER.info("Finished reading log on instance instance={} log={} partitionId={} timeNanos={}",
+                        log.instance().nodeName(), log.name(), partitionId, System.nanoTime() - startTimeNanos);
+        }
+        return null;
+    }
+
+    public StreamScanner build()
+    {
+        // Block on futures to read all CommitLog mutations and pass over to SortedStreamScanner
+        List<PartitionUpdateWrapper> updates = futures.values().stream()
+                .map(future -> FutureUtils.await(future, throwable -> LOGGER.warn("Failed to read instance with error",
+                                                                                  throwable)))
+                .filter(FutureUtils.FutureResult::isSuccess)
+                .map(FutureUtils.FutureResult::value)
+                .filter(Objects::nonNull)
+                .flatMap(Collection::stream)
+                .collect(Collectors.toList());
+        futures.clear();
+
+        schedulePersist();
+
+        Collection<PartitionUpdateWrapper> filtered = filterValidUpdates(updates);
+
+        LOGGER.info("Opened CdcScanner start={} maxAgeMicros={} partitionId={} timeNanos={}",
+                    offsetFilter != null ? offsetFilter.start().getTimestampMicros() : null,
+                    offsetFilter != null ? offsetFilter.maxAgeMicros() : null,
+                    partitionId,
+                    System.nanoTime() - startTimeNanos);
+        return new SortedStreamScanner(table, partitioner, filtered, timeProvider);
+    }
+
+    /**
+     * A stream scanner that is backed by a sorted collection of {@link PartitionUpdateWrapper}
+     */
+    private static class SortedStreamScanner extends AbstractStreamScanner
+    {
+        private final Queue<PartitionUpdateWrapper> updates;
+
+        SortedStreamScanner(@NotNull TableMetadata metadata,
+                            @NotNull Partitioner partitionerType,
+                            @NotNull Collection<PartitionUpdateWrapper> updates,
+                            @NotNull TimeProvider timeProvider)
+        {
+            super(metadata, partitionerType, timeProvider);
+            this.updates = new PriorityQueue<>(PartitionUpdateWrapper::compareTo);
+            this.updates.addAll(updates);
+        }
+
+        @Override
+        UnfilteredPartitionIterator initializePartitions()
+        {
+            return new UnfilteredPartitionIterator()
+            {
+                private PartitionUpdateWrapper next;
+
+                @Override
+                public TableMetadata metadata()
+                {
+                    return metadata;
+                }
+
+                @Override
+                public void close()
+                {
+                    // Do nothing
+                }
+
+                @Override
+                public boolean hasNext()
+                {
+                    if (next == null)
+                    {
+                        next = updates.poll();
+                    }
+                    return next != null;
+                }
+
+                @Override
+                public UnfilteredRowIterator next()
+                {
+                    PartitionUpdate update = next.partitionUpdate();
+                    next = null;
+                    return update.unfilteredIterator();
+                }
+            };
+        }
+
+        @Override
+        public void close()
+        {
+            updates.clear();
+        }
+
+        @Override
+        protected void handleRowTombstone(Row row)
+        {
+            // Prepare clustering data to be consumed the next
+            columnData = new ClusteringColumnDataState(row.clustering());
+            rid.setTimestamp(row.deletion().time().markedForDeleteAt());
+            // Flag was reset at org.apache.cassandra.spark.sparksql.SparkCellIterator.getNext
+            rid.setRowDeletion(true);
+        }
+
+        @Override
+        protected void handlePartitionTombstone(UnfilteredRowIterator partition)
+        {
+            rid.setPartitionKeyCopy(partition.partitionKey().getKey(),
+                                    ReaderUtils.tokenToBigInteger(partition.partitionKey().getToken()));
+            rid.setTimestamp(partition.partitionLevelDeletion().markedForDeleteAt());
+            // Flag was reset at org.apache.cassandra.spark.sparksql.SparkCellIterator.getNext
+            rid.setPartitionDeletion(true);
+        }
+
+        @Override
+        protected void handleCellTombstone()
+        {
+            rid.setValueCopy(null);
+        }
+
+        @Override
+        protected void handleCellTombstoneInComplex(Cell<?> cell)
+        {
+            if (cell.column().type instanceof ListType)
+            {
+                LOGGER.warn("Unable to process element deletions inside a List type. Skipping...");
+                return;
+            }
+
+            CellPath path = cell.path();
+            if (0 < path.size())  // Size can either be 0 (EmptyCellPath) or 1 (SingleItemCellPath)
+            {
+                rid.addCellTombstoneInComplex(path.get(0));
+            }
+        }
+
+        @Override
+        protected void handleRangeTombstone(RangeTombstoneMarker marker)
+        {
+            rid.addRangeTombstoneMarker(new RangeTombstoneMarkerImplementation(marker));
+        }
+    }
+
+    private void schedulePersist()
+    {
+        // Add task listener to persist Watermark on task success
+        TaskContext.get().addTaskCompletionListener(context -> {
+            if (context.isCompleted() && context.fetchFailed().isEmpty())
+            {
+                LOGGER.info("Persisting Watermark on task completion partitionId={}", partitionId);
+                // Once we have read all CommitLogs we can persist the watermark state
+                watermarker.persist(offsetFilter != null ? offsetFilter.maxAgeMicros() : null);
+            }
+            else
+            {
+                LOGGER.warn("Not persisting Watermark due to task failure partitionId={}",
+                            partitionId, context.fetchFailed().get());
+            }
+        });
+    }
+
+    /**
+     * Get rid of invalid updates from the updates
+     *
+     * @param updates, a collection of PartitionUpdateWrappers
+     * @return a new updates without invalid updates
+     */
+    private Collection<PartitionUpdateWrapper> filterValidUpdates(Collection<PartitionUpdateWrapper> updates)
+    {
+        // Only filter if it demands more than 1 replicas to compact
+        if (minimumReplicasPerMutation == 1 || updates.isEmpty())
+        {
+            return updates;
+        }
+
+        Map<PartitionUpdateWrapper, List<PartitionUpdateWrapper>> replicaCopies = updates.stream()
+                .collect(Collectors.groupingBy(update -> update, Collectors.toList()));
+
+        return replicaCopies.values().stream()
+                .filter(this::filter)          // Discard PartitionUpdate without enough replicas
+                .map(update -> update.get(0))  // Deduplicate the valid updates to just singe copy
+                .collect(Collectors.toList());
+    }
+
+    private boolean filter(List<PartitionUpdateWrapper> updates)
+    {
+        return filter(updates, minimumReplicasPerMutation, watermarker, stats);
+    }
+
+    static boolean filter(List<PartitionUpdateWrapper> updates,
+                          int minimumReplicasPerMutation,
+                          Watermarker watermarker,
+                          Stats stats)
+    {
+        if (updates.isEmpty())
+        {
+            throw new IllegalStateException("Should not received empty list of updates");
+        }
+
+        PartitionUpdateWrapper update = updates.get(0);
+        PartitionUpdate partitionUpdate = update.partitionUpdate();
+        int numReplicas = updates.size() + watermarker.replicaCount(update);
+
+        if (numReplicas < minimumReplicasPerMutation)
+        {
+            // Insufficient replica copies to publish, so record replica count and handle on subsequent round
+            LOGGER.warn("Ignore the partition update (partition key: '{}') for this batch "
+                      + "due to insufficient replicas received. {} required {} received.",
+                        partitionUpdate != null ? partitionUpdate.partitionKey() : "unknown",
+                        minimumReplicasPerMutation, numReplicas);
+            watermarker.recordReplicaCount(update, numReplicas);
+            stats.insufficientReplicas(update, updates.size(), minimumReplicasPerMutation);
+            return false;
+        }
+
+        // Sufficient Replica Copies to Publish
+
+        if (updates.stream().anyMatch(watermarker::seenBefore))
+        {
+            // Mutation previously marked as late, now we have sufficient replica copies to publish,
+            // so clear watermark and publish now
+            LOGGER.info("Achieved consistency level for late partition update (partition key: '{}'). {} received.",
+                        partitionUpdate != null ? partitionUpdate.partitionKey() : "unknown", numReplicas);
+            watermarker.untrackReplicaCount(update);
+            stats.lateMutationPublished(update);
+            return true;
+        }
+
+        // We haven't seen this mutation before and achieved CL, so publish
+        stats.publishedMutation(update);
+        return true;
+    }
+
+    public static class CdcScanner implements ISSTableScanner
+    {
+        final TableMetadata tableMetadata;
+        final PartitionUpdate update;
+        UnfilteredRowIterator it;
+
+        public CdcScanner(TableMetadata tableMetadata, PartitionUpdate update)
+        {
+            this.tableMetadata = tableMetadata;
+            this.update = update;
+            this.it = update.unfilteredIterator();
+        }
+
+        @Override
+        public long getLengthInBytes()
+        {
+            return 0;
+        }
+
+        @Override
+        public long getCompressedLengthInBytes()
+        {
+            return 0;
+        }
+
+        @Override
+        public long getCurrentPosition()
+        {
+            return 0;
+        }
+
+        @Override
+        public long getBytesScanned()
+        {
+            return 0;
+        }
+
+        @Override
+        public Set<SSTableReader> getBackingSSTables()
+        {
+            return Collections.emptySet();
+        }
+
+        @Override
+        public TableMetadata metadata()
+        {
+            return tableMetadata;
+        }
+
+        @Override
+        public void close()
+        {
+        }
+
+        @Override
+        public boolean hasNext()
+        {
+            return it != null;
+        }
+
+        @Override
+        public UnfilteredRowIterator next()
+        {
+            UnfilteredRowIterator result = it;
+            it = null;
+            return result;
+        }
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/reader/CompactionStreamScanner.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/reader/CompactionStreamScanner.java
new file mode 100644
index 0000000..7b512ff
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/reader/CompactionStreamScanner.java
@@ -0,0 +1,152 @@
+/*
+ * 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.cassandra.spark.reader;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.UUID;
+import java.util.function.LongPredicate;
+import java.util.stream.Collectors;
+
+import org.apache.cassandra.db.AbstractCompactionController;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.compaction.AbstractCompactionStrategy;
+import org.apache.cassandra.db.compaction.CompactionIterator;
+import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
+import org.apache.cassandra.db.rows.Cell;
+import org.apache.cassandra.db.rows.RangeTombstoneMarker;
+import org.apache.cassandra.db.rows.Row;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.schema.CompactionParams;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.spark.data.partitioner.Partitioner;
+import org.apache.cassandra.spark.utils.IOUtils;
+import org.apache.cassandra.spark.utils.TimeProvider;
+import org.jetbrains.annotations.NotNull;
+
+public class CompactionStreamScanner extends AbstractStreamScanner
+{
+    private final Collection<? extends Scannable> toCompact;
+    private final UUID taskId;
+
+    private PurgingCompactionController controller;
+    private AbstractCompactionStrategy.ScannerList scanners;
+    private CompactionIterator ci;
+
+    CompactionStreamScanner(@NotNull TableMetadata cfMetaData,
+                            @NotNull Partitioner partitionerType,
+                            @NotNull Collection<? extends Scannable> toCompact)
+    {
+        this(cfMetaData, partitionerType, TimeProvider.INSTANCE, toCompact);
+    }
+
+    public CompactionStreamScanner(@NotNull TableMetadata cfMetaData,
+                                   @NotNull Partitioner partitionerType,
+                                   @NotNull TimeProvider timeProvider,
+                                   @NotNull Collection<? extends Scannable> toCompact)
+    {
+        super(cfMetaData, partitionerType, timeProvider);
+        this.toCompact = toCompact;
+        this.taskId = UUID.randomUUID();
+    }
+
+    @Override
+    public void close()
+    {
+        Arrays.asList(controller, scanners, ci)
+              .forEach(IOUtils::closeQuietly);
+    }
+
+    @Override
+    protected void handleRowTombstone(Row row)
+    {
+        throw new IllegalStateException("Row tombstone found, it should have been purged in CompactionIterator");
+    }
+
+    @Override
+    protected void handlePartitionTombstone(UnfilteredRowIterator partition)
+    {
+        throw new IllegalStateException("Partition tombstone found, it should have been purged in CompactionIterator");
+    }
+
+    @Override
+    protected void handleCellTombstone()
+    {
+        throw new IllegalStateException("Cell tombstone found, it should have been purged in CompactionIterator");
+    }
+
+    @Override
+    protected void handleCellTombstoneInComplex(Cell<?> cell)
+    {
+        // Do nothing: to not introduce behavior change to the SBR code path
+    }
+
+    @Override
+    protected void handleRangeTombstone(RangeTombstoneMarker marker)
+    {
+        throw new IllegalStateException("Range tombstone found, it should have been purged in CompactionIterator");
+    }
+
+    @Override
+    UnfilteredPartitionIterator initializePartitions()
+    {
+        int nowInSec = timeProvider.nowInTruncatedSeconds();
+        Keyspace keyspace = Keyspace.openWithoutSSTables(metadata.keyspace);
+        ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore(metadata.name);
+        controller = new PurgingCompactionController(cfStore, CompactionParams.TombstoneOption.NONE);
+        List<ISSTableScanner> scannerList = toCompact.stream()
+                                                     .map(Scannable::scanner)
+                                                     .collect(Collectors.toList());
+        scanners = new AbstractCompactionStrategy.ScannerList(scannerList);
+        ci = new CompactionIterator(OperationType.COMPACTION, scanners.scanners, controller, nowInSec, taskId);
+        return ci;
+    }
+
+    private static class PurgingCompactionController extends AbstractCompactionController implements AutoCloseable
+    {
+        PurgingCompactionController(ColumnFamilyStore cfs, CompactionParams.TombstoneOption tombstoneOption)
+        {
+            super(cfs, Integer.MAX_VALUE, tombstoneOption);
+        }
+
+        @Override
+        public boolean compactingRepaired()
+        {
+            return false;
+        }
+
+        @Override
+        public LongPredicate getPurgeEvaluator(DecoratedKey key)
+        {
+            // Purge all tombstones
+            return time -> true;
+        }
+
+        @Override
+        public void close()
+        {
+        }
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/reader/CompressedRawInputStream.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/reader/CompressedRawInputStream.java
new file mode 100644
index 0000000..03bf2d6
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/reader/CompressedRawInputStream.java
@@ -0,0 +1,286 @@
+/*
+ * 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.cassandra.spark.reader;
+
+import java.io.DataInputStream;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.zip.Checksum;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.spark.data.SSTable;
+import org.apache.cassandra.spark.reader.common.AbstractCompressionMetadata;
+import org.apache.cassandra.spark.reader.common.ChunkCorruptException;
+import org.apache.cassandra.spark.reader.common.RawInputStream;
+import org.apache.cassandra.spark.stats.Stats;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.ChecksumType;
+import org.jetbrains.annotations.Nullable;
+
+public final class CompressedRawInputStream extends RawInputStream
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(CompressedRawInputStream.class);
+
+    @Nullable
+    private final SSTable ssTable;  // Only used for logging/stats
+    private final CompressionMetadata metadata;
+
+    // Used by reBuffer() to escape creating lots of temporary buffers
+    private byte[] compressed;
+    private long currentCompressed = 0;
+
+    // Re-use single checksum object
+    private final Checksum checksum;
+
+    // Raw checksum bytes
+    private final byte[] checksumBytes = new byte[4];
+
+    private CompressedRawInputStream(@Nullable SSTable ssTable,
+                                     DataInputStream source,
+                                     CompressionMetadata metadata,
+                                     Stats stats)
+    {
+        super(source, new byte[metadata.chunkLength()], stats);
+        this.ssTable = ssTable;
+        this.metadata = metadata;
+        this.checksum = ChecksumType.CRC32.newInstance();
+        this.compressed = new byte[metadata.compressor().initialCompressedBufferLength(metadata.chunkLength())];
+    }
+
+    @VisibleForTesting
+    static CompressedRawInputStream fromInputStream(InputStream in,
+                                                    InputStream compressionInfoInputStream,
+                                                    boolean hasCompressedLength) throws IOException
+    {
+        return fromInputStream(null,
+                               new DataInputStream(in),
+                               compressionInfoInputStream,
+                               hasCompressedLength,
+                               Stats.DoNothingStats.INSTANCE);
+    }
+
+    static CompressedRawInputStream fromInputStream(@Nullable SSTable ssTable,
+                                                    DataInputStream dataInputStream,
+                                                    InputStream compressionInfoInputStream,
+                                                    boolean hasCompressedLength,
+                                                    Stats stats) throws IOException
+    {
+        return new CompressedRawInputStream(ssTable,
+                                            dataInputStream,
+                                            CompressionMetadata.fromInputStream(compressionInfoInputStream,
+                                                                                hasCompressedLength),
+                                            stats);
+    }
+
+    @Override
+    public boolean isEOF()
+    {
+        return current >= metadata.getDataLength();
+    }
+
+    private void assertChunkPos(CompressionMetadata.Chunk chunk)
+    {
+        // We may be asked to skip ahead by more than one block
+        assert currentCompressed <= chunk.offset
+            : String.format("Requested chunk at input offset %d is less than current compressed position at %d",
+                            chunk.offset, currentCompressed);
+    }
+
+    private void decompressChunk(CompressionMetadata.Chunk chunk, double crcChance) throws IOException
+    {
+        int checkSumFromChunk;
+
+        assertChunkPos(chunk);
+
+        source.skipBytes((int) (chunk.offset - currentCompressed));
+        currentCompressed = chunk.offset;
+
+        if (compressed.length < chunk.length)
+        {
+            compressed = new byte[chunk.length];
+        }
+
+        if (chunk.length > 0)
+        {
+            try
+            {
+                source.readFully(compressed, 0, chunk.length);
+            }
+            catch (EOFException exception)
+            {
+                throw new IOException(String.format("Failed to read %d bytes from offset %d.",
+                                                    chunk.length, chunk.offset), exception);
+            }
+
+            checkSumFromChunk = source.readInt();
+            stats.readBytes(chunk.length + checksumBytes.length);  // 4 bytes for CRC
+        }
+        else
+        {
+            // Last block; we don't have the length of the last chunk; try to read full buffer length; this
+            // will almost certainly end up reading all of the compressed data; update current chunk length
+            // to the number of the bytes read minus 4 to accommodate for the chunk length field
+            int lastBytesLength = 0;
+            while (true)
+            {
+                if (lastBytesLength >= compressed.length)
+                {
+                    byte[] buffer = new byte[lastBytesLength * 2];
+                    System.arraycopy(compressed, 0, buffer, 0, lastBytesLength);
+                    compressed = buffer;
+                }
+                int readLength = source.read(compressed, lastBytesLength, compressed.length - lastBytesLength);
+                if (readLength < 0)
+                {
+                    break;
+                }
+                stats.readBytes(readLength);
+                lastBytesLength += readLength;
+            }
+
+            chunk.setLength(lastBytesLength - 4);
+
+            // We inadvertently also read the checksum; we need to grab it from the end of the buffer
+            checkSumFromChunk = ByteBufferUtil.toInt(ByteBuffer.wrap(compressed, lastBytesLength - 4, 4));
+        }
+
+        validBufferBytes = metadata.compressor().uncompress(compressed, 0, chunk.length, buffer, 0);
+        stats.decompressedBytes(chunk.length, validBufferBytes);
+
+        if (crcChance > ThreadLocalRandom.current().nextDouble())
+        {
+            checksum.update(compressed, 0, chunk.length);
+
+            if (checkSumFromChunk != (int) checksum.getValue())
+            {
+                throw new ChunkCorruptException("bad chunk " + chunk);
+            }
+
+            // Reset checksum object back to the original (blank) state
+            checksum.reset();
+        }
+
+        currentCompressed += chunk.length + checksumBytes.length;
+
+        alignBufferOffset();
+    }
+
+    /**
+     * Buffer offset is always aligned
+     */
+    private void alignBufferOffset()
+    {
+        // See https://en.wikipedia.org/wiki/Data_structure_alignment#Computing_padding
+        bufferOffset = current & -buffer.length;
+    }
+
+    @Override
+    protected void reBuffer() throws IOException
+    {
+        try
+        {
+            decompressChunk(metadata.chunkAtPosition(current), metadata.crcCheckChance());
+        }
+        catch (IOException exception)
+        {
+            if (ssTable != null)
+            {
+                LOGGER.warn("IOException decompressing SSTable position={} sstable='{}'", current, ssTable, exception);
+                stats.decompressionException(ssTable, exception);
+            }
+            throw exception;
+        }
+    }
+
+    /**
+     * For Compressed input, we can only efficiently skip entire compressed chunks.
+     * Therefore, we:
+     *  1) Skip any uncompressed bytes already buffered. If that's enough to satisfy the skip request, we return.
+     *  2) Count how many whole compressed chunks we can skip to satisfy the uncompressed skip request,
+     *     then skip the compressed bytes at the source InputStream, decrementing the remaining bytes
+     *     by the equivalent uncompressed bytes that have been skipped.
+     *  3) Then we continue to skip any remaining bytes in-memory
+     *     (decompress the next chunk and skip in-memory until we've satisfied the skip request).
+     *
+     * @param count number of uncompressed bytes to skip
+     */
+    @Override
+    public long skip(long count) throws IOException
+    {
+        long precheck = maybeStandardSkip(count);
+        if (precheck >= 0)
+        {
+            return precheck;
+        }
+
+        // Skip any buffered bytes
+        long remaining = count - skipBuffered();
+
+        // We can efficiently skip ahead by 0 or more whole compressed chunks by passing down to the source InputStream
+        long totalCompressedBytes = 0L;
+        long totalDecompressedBytes = 0L;
+        long startCurrent = current;
+        long startCompressed = currentCompressed;
+        long startBufferOffset = bufferOffset;
+        while (totalDecompressedBytes + buffer.length < remaining)  // We can only skip whole chunks
+        {
+            AbstractCompressionMetadata.Chunk chunk = metadata.chunkAtPosition(current);
+            if (chunk.length < 0)
+            {
+                // For the last chunk we don't know the length so reset positions & skip as normal
+                current = startCurrent;
+                currentCompressed = startCompressed;
+                bufferOffset = startBufferOffset;
+                return standardSkip(remaining);
+            }
+
+            assertChunkPos(chunk);
+
+            // Sum total compressed & decompressed bytes we can skip
+            int chunkLength = chunk.length + checksumBytes.length;
+            totalCompressedBytes += (int) (chunk.offset - currentCompressed);
+            currentCompressed = chunk.offset;
+            totalCompressedBytes += chunkLength;
+            currentCompressed += chunkLength;
+            totalDecompressedBytes += buffer.length;
+
+            alignBufferOffset();
+            current += buffer.length;
+        }
+
+        // Skip compressed chunks at the source
+        long skipped = source.skip(totalCompressedBytes);
+        assert skipped == totalCompressedBytes : "Bytes skipped should equal compressed length";
+        remaining -= totalDecompressedBytes;  // Decrement decompressed bytes we have skipped
+
+        // Skip any remaining bytes as normal
+        remaining -= standardSkip(remaining);
+
+        long total = count - remaining;
+        stats.skippedBytes(total);
+        return total;
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/reader/CompressionMetadata.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/reader/CompressionMetadata.java
new file mode 100644
index 0000000..e820180
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/reader/CompressionMetadata.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.cassandra.spark.reader;
+
+import java.io.DataInputStream;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.cassandra.io.compress.ICompressor;
+import org.apache.cassandra.schema.CompressionParams;
+import org.apache.cassandra.spark.reader.common.AbstractCompressionMetadata;
+import org.apache.cassandra.spark.reader.common.BigLongArray;
+
+/**
+ * Holds metadata about compressed file
+ */
+final class CompressionMetadata extends AbstractCompressionMetadata
+{
+
+    private final CompressionParams parameters;
+
+    private CompressionMetadata(long dataLength, BigLongArray chunkOffsets, CompressionParams parameters)
+    {
+        super(dataLength, chunkOffsets);
+        this.parameters = parameters;
+    }
+
+    static CompressionMetadata fromInputStream(InputStream inStream, boolean hasCompressedLength) throws IOException
+    {
+        long dataLength;
+        BigLongArray chunkOffsets;
+
+        DataInputStream inData = new DataInputStream(inStream);
+
+        String compressorName = inData.readUTF();
+        int optionCount = inData.readInt();
+        Map<String, String> options = new HashMap<>(optionCount);
+        for (int option = 0; option < optionCount; ++option)
+        {
+            options.put(inData.readUTF(), inData.readUTF());
+        }
+
+        int chunkLength = inData.readInt();
+        int minCompressRatio = 2147483647;
+        if (hasCompressedLength)
+        {
+            minCompressRatio = inData.readInt();
+        }
+
+        CompressionParams params = new CompressionParams(compressorName, chunkLength, minCompressRatio, options);
+        params.setCrcCheckChance(AbstractCompressionMetadata.CRC_CHECK_CHANCE);
+
+        dataLength = inData.readLong();
+
+        int chunkCount = inData.readInt();
+        chunkOffsets = new BigLongArray(chunkCount);
+
+        for (int chunk = 0; chunk < chunkCount; chunk++)
+        {
+            try
+            {
+                chunkOffsets.set(chunk, inData.readLong());
+            }
+            catch (EOFException exception)
+            {
+                throw new EOFException(String.format("Corrupted compression index: read %d but expected %d chunks.",
+                                                     chunk, chunkCount));
+            }
+        }
+
+        return new CompressionMetadata(dataLength, chunkOffsets, params);
+    }
+
+    ICompressor compressor()
+    {
+        return parameters.getSstableCompressor();
+    }
+
+    @Override
+    protected int chunkLength()
+    {
+        return parameters.chunkLength();
+    }
+
+    @Override
+    protected double crcCheckChance()
+    {
+        return parameters.getCrcCheckChance();
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/reader/CompressionUtils.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/reader/CompressionUtils.java
new file mode 100644
index 0000000..aed6e07
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/reader/CompressionUtils.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.cassandra.spark.reader;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.io.compress.ICompressor;
+import org.apache.cassandra.io.compress.ZstdCompressor;
+
+/**
+ * Util class to make it easy to compress/decompress using any compressor
+ */
+final class CompressionUtils
+{
+    public ICompressor compressor()
+    {
+        return ZstdCompressor.getOrCreate(ZstdCompressor.DEFAULT_COMPRESSION_LEVEL);
+    }
+
+    public ByteBuffer compress(byte[] bytes) throws IOException
+    {
+        ICompressor compressor = compressor();
+        ByteBuffer input = compressor.preferredBufferType().allocate(bytes.length);
+        input.put(bytes);
+        input.flip();
+        return compress(input, compressor);
+    }
+
+    public ByteBuffer compress(ByteBuffer input) throws IOException
+    {
+        return compress(input, compressor());
+    }
+
+    public ByteBuffer compress(ByteBuffer input, ICompressor compressor) throws IOException
+    {
+        int rawSize = input.remaining();  // Store uncompressed length as 4 byte int
+        // 4 extra bytes to store uncompressed length
+        ByteBuffer output = compressor.preferredBufferType().allocate(4 + compressor.initialCompressedBufferLength(rawSize));
+        output.putInt(rawSize);
+        compressor.compress(input, output);
+        output.flip();
+        return output;
+    }
+
+    public ByteBuffer uncompress(byte[] bytes) throws IOException
+    {
+        ICompressor compressor = compressor();
+        ByteBuffer input = compressor.preferredBufferType().allocate(bytes.length);
+        input.put(bytes);
+        input.flip();
+        return uncompress(input, compressor);
+    }
+
+    public ByteBuffer uncompress(ByteBuffer input) throws IOException
+    {
+        return uncompress(input, compressor());
+    }
+
+    public ByteBuffer uncompress(ByteBuffer input, ICompressor compressor) throws IOException
+    {
+        ByteBuffer output = compressor.preferredBufferType().allocate(input.getInt());
+        compressor.uncompress(input, output);
+        output.flip();
+        return output;
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/reader/IndexDbUtils.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/reader/IndexDbUtils.java
new file mode 100644
index 0000000..e289b60
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/reader/IndexDbUtils.java
@@ -0,0 +1,173 @@
+/*
+ * 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.cassandra.spark.reader;
+
+import java.io.DataInputStream;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.bridge.TokenRange;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.sstable.IndexSummary;
+import org.apache.cassandra.spark.data.SSTable;
+import org.apache.cassandra.spark.stats.Stats;
+import org.apache.cassandra.spark.utils.ByteBufferUtils;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Helper methods for reading the Index.db SSTable file component
+ */
+final class IndexDbUtils
+{
+    private IndexDbUtils()
+    {
+        throw new IllegalStateException(getClass() + " is static utility class and shall not be instantiated");
+    }
+
+    @Nullable
+    public static Long findDataDbOffset(@NotNull IndexSummary indexSummary,
+                                        @NotNull TokenRange range,
+                                        @NotNull IPartitioner partitioner,
+                                        @NotNull SSTable ssTable,
+                                        @NotNull Stats stats) throws IOException
+    {
+        long searchStartOffset = SummaryDbUtils.findIndexOffsetInSummary(indexSummary, partitioner, range.lowerEndpoint());
+
+        // Open the Index.db, skip to nearest offset found in Summary.db and find start & end offset for the Data.db file
+        return findDataDbOffset(range, partitioner, ssTable, stats, searchStartOffset);
+    }
+
+    @Nullable
+    public static Long findDataDbOffset(@NotNull TokenRange range,
+                                        @NotNull IPartitioner partitioner,
+                                        @NotNull SSTable ssTable,
+                                        @NotNull Stats stats,
+                                        long searchStartOffset) throws IOException
+    {
+        try (InputStream is = ssTable.openPrimaryIndexStream())
+        {
+            return findIndexOffset(is, partitioner, range, stats, searchStartOffset);
+        }
+    }
+
+    /**
+     * Find the first Data.db offset in the Index.db file for a given token range,
+     * using the approximate start offset found in the Summary.db file to seek ahead to the nearest position in the Index.db file
+     *
+     * @param is                the input stream on the Index.db file
+     * @param partitioner       Cassandra partitioner
+     * @param range             the range we are trying to find
+     * @param stats             stats instance
+     * @param searchStartOffset the Index.db approximate start offset read from the Summary.db sample file
+     * @return the index offset into the Data.db file for the first partition greater than or equal to the token, or null if cannot find
+     * @throws IOException IOException reading Index.db file
+     */
+    @Nullable
+    static Long findIndexOffset(@Nullable InputStream is,
+                                @NotNull IPartitioner partitioner,
+                                @NotNull TokenRange range,
+                                @NotNull Stats stats,
+                                long searchStartOffset) throws IOException
+    {
+        if (is == null)
+        {
+            return null;
+        }
+
+        try
+        {
+            // Skip to Index.db offset found in Summary.db file
+            DataInputStream in = new DataInputStream(is);
+            ByteBufferUtils.skipFully(in, searchStartOffset);
+
+            return findStartOffset(in, partitioner, range, stats);
+        }
+        catch (EOFException ignore)
+        {
+            // We can possibly reach EOF before start has been found, which is fine
+        }
+
+        return null;
+    }
+
+    /**
+     * Find and return Data.db offset for first overlapping partition
+     *
+     * @param in          Index.db DataInputStream
+     * @param partitioner partitioner
+     * @param range       Spark worker token range
+     * @param stats       stats instance
+     * @return start offset into the Data.db file for the first overlapping partition
+     * @throws IOException IOException reading Index.db file
+     */
+    static long findStartOffset(@NotNull DataInputStream in,
+                                @NotNull IPartitioner partitioner,
+                                @NotNull TokenRange range,
+                                @NotNull Stats stats) throws IOException
+    {
+        BigInteger keyToken;
+        long previous = 0L;
+        // CHECKSTYLE IGNORE: An idiomatic way to read input streams
+        while (isLessThan(keyToken = readNextToken(partitioner, in, stats), range))
+        {
+            // Keep skipping until we find first partition overlapping with Spark token range
+            previous = ReaderUtils.readPosition(in);
+            ReaderUtils.skipPromotedIndex(in);
+        }
+        assert range.lowerEndpoint().compareTo(keyToken) <= 0;
+        // Found first token that overlaps with Spark token range because we passed the target
+        // by skipping the promoted index, we use the previously-read position as start
+        return previous;
+    }
+
+    /**
+     * @param keyToken key token read from Index.db
+     * @param range    spark worker token range
+     * @return true if keyToken is less than the range lower bound
+     */
+    static boolean isLessThan(@NotNull BigInteger keyToken, @NotNull TokenRange range)
+    {
+        return keyToken.compareTo(range.lowerEndpoint()) < 0;
+    }
+
+    /**
+     * Read partition key, use partitioner to hash and return token as BigInteger
+     *
+     * @param partitioner partitioner
+     * @param in          Index.db DataInputStream
+     * @param stats       stats instance
+     * @return token as BigInteger
+     * @throws IOException IOException reading Index.db file
+     */
+    static BigInteger readNextToken(@NotNull IPartitioner partitioner,
+                                    @NotNull DataInputStream in,
+                                    @NotNull Stats stats) throws IOException
+    {
+        ByteBuffer key = ByteBufferUtil.readWithShortLength(in);
+        BigInteger token = ReaderUtils.tokenToBigInteger(partitioner.decorateKey(key).getToken());
+        stats.readPartitionIndexDb((ByteBuffer) key.rewind(), token);
+        return token;
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/reader/ReaderUtils.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/reader/ReaderUtils.java
new file mode 100644
index 0000000..35c29b5
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/reader/ReaderUtils.java
@@ -0,0 +1,547 @@
+/*
+ * 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.cassandra.spark.reader;
+
+import java.io.DataInputStream;
+import java.io.EOFException;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.EnumMap;
+import java.util.EnumSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.zip.CRC32;
+
+import org.apache.cassandra.db.Clustering;
+import org.apache.cassandra.db.ClusteringPrefix;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.SerializationHeader;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.ByteBufferAccessor;
+import org.apache.cassandra.db.marshal.CompositeType;
+import org.apache.cassandra.db.marshal.TypeParser;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.db.rows.EncodingStats;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.dht.RandomPartitioner;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.io.sstable.metadata.MetadataComponent;
+import org.apache.cassandra.io.sstable.metadata.MetadataType;
+import org.apache.cassandra.io.sstable.metadata.ValidationMetadata;
+import org.apache.cassandra.io.util.DataInputBuffer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.spark.data.SSTable;
+import org.apache.cassandra.spark.sparksql.filters.PartitionKeyFilter;
+import org.apache.cassandra.spark.utils.ByteBufferUtils;
+import org.apache.cassandra.utils.BloomFilter;
+import org.apache.cassandra.utils.BloomFilterSerializer;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.vint.VIntCoding;
+import org.jetbrains.annotations.NotNull;
+
+import static org.apache.cassandra.utils.FBUtilities.updateChecksumInt;
+
+@SuppressWarnings("WeakerAccess")
+public final class ReaderUtils
+{
+    private static final int CHECKSUM_LENGTH = 4;  // CRC32
+    private static final Constructor<?> SERIALIZATION_HEADER =
+            Arrays.stream(SerializationHeader.Component.class.getDeclaredConstructors())
+                  .filter(constructor -> constructor.getParameterCount() == 5)
+                  .findFirst()
+                  .orElseThrow(() -> new RuntimeException("Could not find SerializationHeader.Component constructor"));
+    public static final ByteBuffer SUPER_COLUMN_MAP_COLUMN = ByteBufferUtil.EMPTY_BYTE_BUFFER;
+
+    static
+    {
+        SERIALIZATION_HEADER.setAccessible(true);
+    }
+
+    private ReaderUtils()
+    {
+        throw new IllegalStateException(getClass() + " is static utility class and shall not be instantiated");
+    }
+
+    public static long tokenToLong(Token token)
+    {
+        if (token instanceof Murmur3Partitioner.LongToken)
+        {
+            return (long) token.getTokenValue();
+        }
+        if (token instanceof RandomPartitioner.BigIntegerToken)
+        {
+            return ((RandomPartitioner.BigIntegerToken) token).getTokenValue().longValue();
+        }
+
+        throw new UnsupportedOperationException("Unexpected token type: " + token.getClass().getName());
+    }
+
+    public static BigInteger tokenToBigInteger(Token token)
+    {
+        if (token instanceof Murmur3Partitioner.LongToken)
+        {
+            return BigInteger.valueOf((long) token.getTokenValue());
+        }
+        if (token instanceof RandomPartitioner.BigIntegerToken)
+        {
+            return ((RandomPartitioner.BigIntegerToken) token).getTokenValue();
+        }
+
+        throw new UnsupportedOperationException("Unexpected token type: " + token.getClass().getName());
+    }
+
+    static ByteBuffer encodeCellName(TableMetadata metadata,
+                                     ClusteringPrefix clustering,
+                                     ByteBuffer columnName,
+                                     ByteBuffer collectionElement)
+    {
+        boolean isStatic = clustering == Clustering.STATIC_CLUSTERING;
+
+        if (!TableMetadata.Flag.isCompound(metadata.flags))
+        {
+            if (isStatic)
+            {
+                return columnName;
+            }
+
+            assert clustering.size() == 1 : "Expected clustering size to be 1, but was " + clustering.size();
+            return clustering.bufferAt(0);
+        }
+
+        // We use comparator.size() rather than clustering.size() because of static clusterings
+        int clusteringSize = metadata.comparator.size();
+        int size = clusteringSize + (TableMetadata.Flag.isDense(metadata.flags) ? 0 : 1)
+                                  + (collectionElement == null ? 0 : 1);
+        if (TableMetadata.Flag.isSuper(metadata.flags))
+        {
+            size = clusteringSize + 1;
+        }
+
+        ByteBuffer[] values = new ByteBuffer[size];
+        for (int index = 0; index < clusteringSize; index++)
+        {
+            if (isStatic)
+            {
+                values[index] = ByteBufferUtil.EMPTY_BYTE_BUFFER;
+                continue;
+            }
+
+            ByteBuffer value = clustering.bufferAt(index);
+            // We can have null (only for dense compound tables for backward compatibility reasons),
+            // but that means we're done and should stop there as far as building the composite is concerned
+            if (value == null)
+            {
+                return CompositeType.build(ByteBufferAccessor.instance, Arrays.copyOfRange(values, 0, index));
+            }
+
+            values[index] = value;
+        }
+
+        if (TableMetadata.Flag.isSuper(metadata.flags))
+        {
+            // We need to set the "column" (in thrift terms) name, i.e. the value corresponding to the subcomparator.
+            // What it is depends on whether this is a cell for a declared "static" column
+            // or a "dynamic" column part of the super-column internal map.
+            assert columnName != null;  // This should never be null for supercolumns, see decodeForSuperColumn() above
+            values[clusteringSize] = columnName.equals(SUPER_COLUMN_MAP_COLUMN)
+                                     ? collectionElement
+                                     : columnName;
+        }
+        else
+        {
+            if (!TableMetadata.Flag.isDense(metadata.flags))
+            {
+                values[clusteringSize] = columnName;
+            }
+            if (collectionElement != null)
+            {
+                values[clusteringSize + 1] = collectionElement;
+            }
+        }
+
+        return CompositeType.build(ByteBufferAccessor.instance, isStatic, values);
+    }
+
+    static Pair<DecoratedKey, DecoratedKey> keysFromIndex(@NotNull TableMetadata metadata,
+                                                          @NotNull SSTable ssTable) throws IOException
+    {
+        try (InputStream primaryIndex = ssTable.openPrimaryIndexStream())
+        {
+            if (primaryIndex != null)
+            {
+                IPartitioner partitioner = metadata.partitioner;
+                Pair<ByteBuffer, ByteBuffer> keys = readPrimaryIndex(primaryIndex, true, Collections.emptyList());
+                return Pair.create(partitioner.decorateKey(keys.left), partitioner.decorateKey(keys.right));
+            }
+        }
+        return Pair.create(null, null);
+    }
+
+    static boolean anyFilterKeyInIndex(@NotNull SSTable ssTable,
+                                       @NotNull List<PartitionKeyFilter> filters) throws IOException
+    {
+        if (filters.isEmpty())
+        {
+            return false;
+        }
+
+        try (InputStream primaryIndex = ssTable.openPrimaryIndexStream())
+        {
+            if (primaryIndex != null)
+            {
+                Pair<ByteBuffer, ByteBuffer> keys = readPrimaryIndex(primaryIndex, false, filters);
+                if (keys.left != null || keys.right != null)
+                {
+                    return false;
+                }
+            }
+        }
+        return true;
+    }
+
+    static Map<MetadataType, MetadataComponent> deserializeStatsMetadata(SSTable ssTable,
+                                                                         Descriptor descriptor) throws IOException
+    {
+        try (InputStream statsStream = ssTable.openStatsStream())
+        {
+            return deserializeStatsMetadata(statsStream,
+                                            EnumSet.of(MetadataType.VALIDATION, MetadataType.STATS, MetadataType.HEADER),
+                                            descriptor);
+        }
+    }
+
+    /**
+     * Deserialize Statistics.db file to pull out metadata components needed for SSTable deserialization
+     *
+     * @param is            input stream for Statistics.db file
+     * @param selectedTypes enum of MetadataType to deserialize
+     * @param descriptor    SSTable file descriptor
+     * @return map of MetadataComponent for each requested MetadataType
+     * @throws IOException
+     */
+    static Map<MetadataType, MetadataComponent> deserializeStatsMetadata(InputStream is,
+                                                                         EnumSet<MetadataType> selectedTypes,
+                                                                         Descriptor descriptor) throws IOException
+    {
+        DataInputStream in = new DataInputPlus.DataInputStreamPlus(is);
+        boolean isChecksummed = descriptor.version.hasMetadataChecksum();
+        CRC32 crc = new CRC32();
+
+        int count = in.readInt();
+        updateChecksumInt(crc, count);
+        maybeValidateChecksum(crc, in, descriptor);
+
+        int[] ordinals = new int[count];
+        int[] offsets = new int[count];
+        int[] lengths = new int[count];
+
+        for (int index = 0; index < count; index++)
+        {
+            ordinals[index] = in.readInt();
+            updateChecksumInt(crc, ordinals[index]);
+
+            offsets[index] = in.readInt();
+            updateChecksumInt(crc, offsets[index]);
+        }
+        maybeValidateChecksum(crc, in, descriptor);
+
+        for (int index = 0; index < count - 1; index++)
+        {
+            lengths[index] = offsets[index + 1] - offsets[index];
+        }
+
+        MetadataType[] allMetadataTypes = MetadataType.values();
+        Map<MetadataType, MetadataComponent> components = new EnumMap<>(MetadataType.class);
+        for (int index = 0; index < count - 1; index++)
+        {
+            MetadataType type = allMetadataTypes[ordinals[index]];
+
+            if (!selectedTypes.contains(type))
+            {
+                in.skipBytes(lengths[index]);
+                continue;
+            }
+
+            byte[] bytes = new byte[isChecksummed ? lengths[index] - CHECKSUM_LENGTH : lengths[index]];
+            in.readFully(bytes);
+
+            crc.reset();
+            crc.update(bytes);
+            maybeValidateChecksum(crc, in, descriptor);
+
+            components.put(type, deserializeMetadataComponent(descriptor.version, bytes, type));
+        }
+
+        MetadataType type = allMetadataTypes[ordinals[count - 1]];
+        if (!selectedTypes.contains(type))
+        {
+            return components;
+        }
+
+        // We do not have in.bytesRemaining() (as in FileDataInput),
+        // so need to read remaining bytes to get final component
+        byte[] remainingBytes = ByteBufferUtils.readRemainingBytes(in, 256);
+        byte[] bytes;
+        if (descriptor.version.hasMetadataChecksum())
+        {
+            ByteBuffer buffer = ByteBuffer.wrap(remainingBytes);
+            int length = buffer.remaining() - 4;
+            bytes = new byte[length];
+            buffer.get(bytes, 0, length);
+            crc.reset();
+            crc.update(bytes);
+            validateChecksum(crc, buffer.getInt(), descriptor);
+        }
+        else
+        {
+            bytes = remainingBytes;
+        }
+
+        components.put(type, deserializeMetadataComponent(descriptor.version, bytes, type));
+
+        return components;
+    }
+
+    private static void maybeValidateChecksum(CRC32 crc, DataInputStream in, Descriptor descriptor) throws IOException
+    {
+        if (descriptor.version.hasMetadataChecksum())
+        {
+            validateChecksum(crc, in.readInt(), descriptor);
+        }
+    }
+
+    private static void validateChecksum(CRC32 crc, int expectedChecksum, Descriptor descriptor)
+    {
+        int actualChecksum = (int) crc.getValue();
+
+        if (actualChecksum != expectedChecksum)
+        {
+            String filename = descriptor.filenameFor(Component.STATS);
+            throw new CorruptSSTableException(new IOException("Checksums do not match for " + filename), filename);
+        }
+    }
+
+    private static MetadataComponent deserializeValidationMetaData(@NotNull DataInputBuffer in) throws IOException
+    {
+        return new ValidationMetadata(in.readUTF(), in.readDouble());
+    }
+
+    private static MetadataComponent deserializeMetadataComponent(@NotNull Version version,
+                                                                  @NotNull byte[] buffer,
+                                                                  @NotNull MetadataType type) throws IOException
+    {
+        DataInputBuffer in = new DataInputBuffer(buffer);
+        if (type == MetadataType.HEADER)
+        {
+            return deserializeSerializationHeader(in);
+        }
+        else if (type == MetadataType.VALIDATION)
+        {
+            return deserializeValidationMetaData(in);
+        }
+        return type.serializer.deserialize(version, in);
+    }
+
+    private static MetadataComponent deserializeSerializationHeader(@NotNull DataInputBuffer in) throws IOException
+    {
+        // We need to deserialize data type class names using shaded package names
+        EncodingStats stats = EncodingStats.serializer.deserialize(in);
+        AbstractType<?> keyType = readType(in);
+        int size = (int) in.readUnsignedVInt();
+        List<AbstractType<?>> clusteringTypes = new ArrayList<>(size);
+
+        for (int index = 0; index < size; ++index)
+        {
+            clusteringTypes.add(readType(in));
+        }
+
+        Map<ByteBuffer, AbstractType<?>> staticColumns = new LinkedHashMap<>();
+        Map<ByteBuffer, AbstractType<?>> regularColumns = new LinkedHashMap<>();
+        readColumnsWithType(in, staticColumns);
+        readColumnsWithType(in, regularColumns);
+
+        try
+        {
+            // TODO: We should expose this code in Cassandra to make it easier to do this with unit tests in Cassandra
+            return (SerializationHeader.Component) SERIALIZATION_HEADER.newInstance(keyType,
+                                                                                    clusteringTypes,
+                                                                                    staticColumns,
+                                                                                    regularColumns,
+                                                                                    stats);
+        }
+        catch (InstantiationException | IllegalAccessException | InvocationTargetException exception)
+        {
+            throw new RuntimeException(exception);
+        }
+    }
+
+    private static void readColumnsWithType(@NotNull DataInputPlus in,
+                                            @NotNull Map<ByteBuffer, AbstractType<?>> typeMap) throws IOException
+    {
+        int length = (int) in.readUnsignedVInt();
+        for (int index = 0; index < length; index++)
+        {
+            ByteBuffer name = ByteBufferUtil.readWithVIntLength(in);
+            typeMap.put(name, readType(in));
+        }
+    }
+
+    private static AbstractType<?> readType(@NotNull DataInputPlus in) throws IOException
+    {
+        return TypeParser.parse(UTF8Type.instance.compose(ByteBufferUtil.readWithVIntLength(in)));
+    }
+
+    /**
+     * Read primary Index.db file, read through all partitions to get first and last partition key
+     *
+     * @param primaryIndex input stream for Index.db file
+     * @return pair of first and last decorated keys
+     * @throws IOException
+     */
+    @SuppressWarnings("InfiniteLoopStatement")
+    static Pair<ByteBuffer, ByteBuffer> readPrimaryIndex(@NotNull InputStream primaryIndex,
+                                                         boolean readFirstLastKey,
+                                                         @NotNull List<PartitionKeyFilter> filters) throws IOException
+    {
+        ByteBuffer firstKey = null;
+        ByteBuffer lastKey = null;
+        try (DataInputStream dis = new DataInputStream(primaryIndex))
+        {
+            byte[] last = null;
+            try
+            {
+                while (true)
+                {
+                    int length = dis.readUnsignedShort();
+                    byte[] buffer = new byte[length];
+                    dis.readFully(buffer);
+                    if (firstKey == null)
+                    {
+                        firstKey = ByteBuffer.wrap(buffer);
+                    }
+                    last = buffer;
+                    ByteBuffer key = ByteBuffer.wrap(last);
+                    if (!readFirstLastKey && filters.stream().anyMatch(filter -> filter.filter(key)))
+                    {
+                        return Pair.create(null, null);
+                    }
+
+                    // Read position and skip promoted index
+                    skipRowIndexEntry(dis);
+                }
+            }
+            catch (EOFException ignored)
+            {
+            }
+
+            if (last != null)
+            {
+                lastKey = ByteBuffer.wrap(last);
+            }
+        }
+
+        return Pair.create(firstKey, lastKey);
+    }
+
+    static void skipRowIndexEntry(DataInputStream dis) throws IOException
+    {
+        readPosition(dis);
+        skipPromotedIndex(dis);
+    }
+
+    static int vIntSize(long value)
+    {
+        return VIntCoding.computeUnsignedVIntSize(value);
+    }
+
+    static void writePosition(long value, ByteBuffer buffer)
+    {
+        VIntCoding.writeUnsignedVInt(value, buffer);
+    }
+
+    static long readPosition(DataInputStream dis) throws IOException
+    {
+        return VIntCoding.readUnsignedVInt(dis);
+    }
+
+    static void skipPromotedIndex(DataInputStream dis) throws IOException
+    {
+        int size = (int) VIntCoding.readUnsignedVInt(dis);
+        if (size > 0)
+        {
+            ByteBufferUtils.skipBytesFully(dis, size);
+        }
+    }
+
+    static List<PartitionKeyFilter> filterKeyInBloomFilter(
+            @NotNull SSTable ssTable,
+            @NotNull IPartitioner partitioner,
+            Descriptor descriptor,
+            @NotNull List<PartitionKeyFilter> partitionKeyFilters) throws IOException
+    {
+        try
+        {
+            BloomFilter bloomFilter = SSTableCache.INSTANCE.bloomFilter(ssTable, descriptor);
+            return partitionKeyFilters.stream()
+                                      .filter(filter -> bloomFilter.isPresent(partitioner.decorateKey(filter.key())))
+                                      .collect(Collectors.toList());
+        }
+        catch (Exception exception)
+        {
+            if (exception instanceof FileNotFoundException)
+            {
+                return partitionKeyFilters;
+            }
+            throw exception;
+        }
+    }
+
+    static BloomFilter readFilter(@NotNull SSTable ssTable, boolean hasOldBfFormat) throws IOException
+    {
+        try (InputStream filterStream = ssTable.openFilterStream())
+        {
+            if (filterStream != null)
+            {
+                try (DataInputStream dis = new DataInputStream(filterStream))
+                {
+                    return BloomFilterSerializer.deserialize(dis, hasOldBfFormat);
+                }
+            }
+        }
+        throw new FileNotFoundException();
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/reader/SSTableCache.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/reader/SSTableCache.java
new file mode 100644
index 0000000..f13d4d5
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/reader/SSTableCache.java
@@ -0,0 +1,155 @@
+/*
+ * 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.cassandra.spark.reader;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.metadata.MetadataComponent;
+import org.apache.cassandra.io.sstable.metadata.MetadataType;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.spark.data.SSTable;
+import org.apache.cassandra.spark.utils.ThrowableUtils;
+import org.apache.cassandra.utils.BloomFilter;
+import org.apache.cassandra.utils.Pair;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Basic cache to reduce wasteful requests on the DataLayer for cacheable SSTable metadata,
+ * useful when running many Spark tasks on the same Spark worker
+ */
+@SuppressWarnings("UnstableApiUsage")
+public class SSTableCache
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(SSTableCache.class);
+
+    public static final SSTableCache INSTANCE = new SSTableCache();
+    private final Cache<SSTable, SummaryDbUtils.Summary>             summary = buildCache(propOrDefault("sbr.cache.summary.maxEntries", 4096),
+                                                                                          propOrDefault("sbr.cache.summary.expireAfterMins", 15));
+    private final Cache<SSTable, Pair<DecoratedKey, DecoratedKey>>     index = buildCache(propOrDefault("sbr.cache.index.maxEntries", 128),
+                                                                                          propOrDefault("sbr.cache.index.expireAfterMins", 60));
+    private final Cache<SSTable, Map<MetadataType, MetadataComponent>> stats = buildCache(propOrDefault("sbr.cache.stats.maxEntries", 16384),
+                                                                                          propOrDefault("sbr.cache.stats.expireAfterMins", 60));
+    private final Cache<SSTable, BloomFilter>                         filter = buildCache(propOrDefault("sbr.cache.filter.maxEntries", 16384),
+                                                                                          propOrDefault("sbr.cache.filter.expireAfterMins", 60));
+
+    private static int propOrDefault(String name, int defaultValue)
+    {
+        String str = System.getProperty(name);
+        if (str != null)
+        {
+            try
+            {
+                return Integer.parseInt(str);
+            }
+            catch (NumberFormatException exception)
+            {
+                LOGGER.error("NumberFormatException for prop {} ", name, exception);
+            }
+        }
+        return defaultValue;
+    }
+
+    private <T> Cache<SSTable, T> buildCache(int size, int expireAfterMins)
+    {
+        return CacheBuilder.newBuilder()
+                           .expireAfterAccess(expireAfterMins, TimeUnit.MINUTES)
+                           .maximumSize(size)
+                           .build();
+    }
+
+    public SummaryDbUtils.Summary keysFromSummary(@NotNull TableMetadata metadata,
+                                                  @NotNull SSTable ssTable) throws IOException
+    {
+        return get(summary, ssTable, () -> SummaryDbUtils.readSummary(metadata, ssTable));
+    }
+
+    public Pair<DecoratedKey, DecoratedKey> keysFromIndex(@NotNull TableMetadata metadata,
+                                                          @NotNull SSTable ssTable) throws IOException
+    {
+        return get(index, ssTable, () -> ReaderUtils.keysFromIndex(metadata, ssTable));
+    }
+
+    public Map<MetadataType, MetadataComponent> componentMapFromStats(@NotNull SSTable ssTable,
+                                                                      Descriptor descriptor) throws IOException
+    {
+        return get(stats, ssTable, () -> ReaderUtils.deserializeStatsMetadata(ssTable, descriptor));
+    }
+
+    public BloomFilter bloomFilter(@NotNull SSTable ssTable, Descriptor descriptor) throws IOException
+    {
+        return get(filter, ssTable, () -> ReaderUtils.readFilter(ssTable, descriptor.version.hasOldBfFormat()));
+    }
+
+    boolean containsSummary(@NotNull SSTable ssTable)
+    {
+        return contains(summary, ssTable);
+    }
+
+    boolean containsIndex(@NotNull SSTable ssTable)
+    {
+        return contains(index, ssTable);
+    }
+
+    boolean containsStats(@NotNull SSTable ssTable)
+    {
+        return contains(stats, ssTable);
+    }
+
+    boolean containsFilter(@NotNull SSTable ssTable)
+    {
+        return contains(filter, ssTable);
+    }
+
+    private static <T> boolean contains(@NotNull Cache<SSTable, T> cache, @NotNull SSTable ssTable)
+    {
+        return cache.getIfPresent(ssTable) != null;
+    }
+
+    private static <T> T get(@NotNull Cache<SSTable, T> cache,
+                             @NotNull SSTable ssTable,
+                             @NotNull Callable<T> callable) throws IOException
+    {
+        try
+        {
+            return cache.get(ssTable, callable);
+        }
+        catch (ExecutionException exception)
+        {
+            throw toIOException(exception);
+        }
+    }
+
+    private static IOException toIOException(Throwable throwable)
+    {
+        IOException ioException = ThrowableUtils.rootCause(throwable, IOException.class);
+        return ioException != null ? ioException : new IOException(ThrowableUtils.rootCause(throwable));
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/reader/SSTableReader.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/reader/SSTableReader.java
new file mode 100644
index 0000000..a5bbc36
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/reader/SSTableReader.java
@@ -0,0 +1,875 @@
+/*
+ * 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.cassandra.spark.reader;
+
+import java.io.DataInputStream;
+import java.io.EOFException;
+import java.io.File;
+import java.io.IOError;
+import java.io.IOException;
+import java.io.InputStream;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Streams;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.bridge.TokenRange;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.RegularAndStaticColumns;
+import org.apache.cassandra.db.SerializationHeader;
+import org.apache.cassandra.db.UnfilteredDeserializer;
+import org.apache.cassandra.db.filter.ColumnFilter;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.db.rows.DeserializationHelper;
+import org.apache.cassandra.db.rows.EncodingStats;
+import org.apache.cassandra.db.rows.Row;
+import org.apache.cassandra.db.rows.Unfiltered;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.sstable.IndexSummary;
+import org.apache.cassandra.io.sstable.SSTableSimpleIterator;
+import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.io.sstable.metadata.MetadataComponent;
+import org.apache.cassandra.io.sstable.metadata.MetadataType;
+import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
+import org.apache.cassandra.io.sstable.metadata.ValidationMetadata;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.DroppedColumn;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.ActiveRepairService;
+import org.apache.cassandra.spark.data.SSTable;
+import org.apache.cassandra.spark.reader.common.RawInputStream;
+import org.apache.cassandra.spark.reader.common.SSTableStreamException;
+import org.apache.cassandra.spark.sparksql.filters.PartitionKeyFilter;
+import org.apache.cassandra.spark.sparksql.filters.PruneColumnFilter;
+import org.apache.cassandra.spark.sparksql.filters.SparkRangeFilter;
+import org.apache.cassandra.spark.stats.Stats;
+import org.apache.cassandra.spark.utils.ByteBufferUtils;
+import org.apache.cassandra.spark.utils.ThrowableUtils;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.Pair;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+@SuppressWarnings("unused")
+public class SSTableReader implements SparkSSTableReader, Scannable
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(SSTableReader.class);
+
+    private final TableMetadata metadata;
+    @NotNull
+    private final SSTable ssTable;
+    private final StatsMetadata statsMetadata;
+    @NotNull
+    private final Version version;
+    @NotNull
+    private final DecoratedKey first;
+    @NotNull
+    private final DecoratedKey last;
+    @NotNull
+    private final BigInteger firstToken;
+    @NotNull
+    private final BigInteger lastToken;
+    private final SerializationHeader header;
+    private final DeserializationHelper helper;
+    @NotNull
+    private final AtomicReference<SSTableStreamReader> reader = new AtomicReference<>(null);
+    @Nullable
+    private final SparkRangeFilter sparkRangeFilter;
+    @NotNull
+    private final List<PartitionKeyFilter> partitionKeyFilters;
+    @NotNull
+    private final Stats stats;
+    @Nullable
+    private Long startOffset = null;
+    private Long openedNanos = null;
+    @NotNull
+    private final Function<StatsMetadata, Boolean> isRepaired;
+
+    public static class Builder
+    {
+        @NotNull
+        final TableMetadata metadata;
+        @NotNull
+        final SSTable ssTable;
+        @Nullable
+        PruneColumnFilter columnFilter = null;
+        boolean readIndexOffset = true;
+        @NotNull
+        Stats stats = Stats.DoNothingStats.INSTANCE;
+        boolean useIncrementalRepair = true;
+        boolean isRepairPrimary = false;
+        Function<StatsMetadata, Boolean> isRepaired = stats -> stats.repairedAt != ActiveRepairService.UNREPAIRED_SSTABLE;
+        @Nullable
+        SparkRangeFilter sparkRangeFilter = null;
+        @NotNull
+        final List<PartitionKeyFilter> partitionKeyFilters = new ArrayList<>();
+
+        Builder(@NotNull TableMetadata metadata, @NotNull SSTable ssTable)
+        {
+            this.metadata = metadata;
+            this.ssTable = ssTable;
+        }
+
+        public Builder withSparkRangeFilter(@Nullable SparkRangeFilter sparkRangeFilter)
+        {
+            this.sparkRangeFilter = sparkRangeFilter;
+            return this;
+        }
+
+        public Builder withPartitionKeyFilters(@NotNull Collection<PartitionKeyFilter> partitionKeyFilters)
+        {
+            this.partitionKeyFilters.addAll(partitionKeyFilters);
+            return this;
+        }
+
+        public Builder withPartitionKeyFilter(@NotNull PartitionKeyFilter partitionKeyFilter)
+        {
+            partitionKeyFilters.add(partitionKeyFilter);
+            return this;
+        }
+
+        public Builder withColumnFilter(@Nullable PruneColumnFilter columnFilter)
+        {
+            this.columnFilter = columnFilter;
+            return this;
+        }
+
+        public Builder withReadIndexOffset(boolean readIndexOffset)
+        {
+            this.readIndexOffset = readIndexOffset;
+            return this;
+        }
+
+        public Builder withStats(@NotNull Stats stats)
+        {
+            this.stats = stats;
+            return this;
+        }
+
+        public Builder useIncrementalRepair(boolean useIncrementalRepair)
+        {
+            this.useIncrementalRepair = useIncrementalRepair;
+            return this;
+        }
+
+        public Builder isRepairPrimary(boolean isRepairPrimary)
+        {
+            this.isRepairPrimary = isRepairPrimary;
+            return this;
+        }
+
+        public Builder withIsRepairedFunction(Function<StatsMetadata, Boolean> isRepaired)
+        {
+            this.isRepaired = isRepaired;
+            return this;
+        }
+
+        public SSTableReader build() throws IOException
+        {
+            return new SSTableReader(metadata,
+                                     ssTable,
+                                     sparkRangeFilter,
+                                     partitionKeyFilters,
+                                     columnFilter,
+                                     readIndexOffset,
+                                     stats,
+                                     useIncrementalRepair,
+                                     isRepairPrimary,
+                                     isRepaired);
+        }
+    }
+
+    public static Builder builder(@NotNull TableMetadata metadata, @NotNull SSTable ssTable)
+    {
+        return new Builder(metadata, ssTable);
+    }
+
+    // CHECKSTYLE IGNORE: Constructor with many parameters
+    public SSTableReader(@NotNull TableMetadata metadata,
+                         @NotNull SSTable ssTable,
+                         @Nullable SparkRangeFilter sparkRangeFilter,
+                         @NotNull List<PartitionKeyFilter> partitionKeyFilters,
+                         @Nullable PruneColumnFilter columnFilter,
+                         boolean readIndexOffset,
+                         @NotNull Stats stats,
+                         boolean useIncrementalRepair,
+                         boolean isRepairPrimary,
+                         @NotNull Function<StatsMetadata, Boolean> isRepaired) throws IOException
+    {
+        long startTimeNanos = System.nanoTime();
+        long now;
+        this.ssTable = ssTable;
+        this.stats = stats;
+        this.isRepaired = isRepaired;
+        this.sparkRangeFilter = sparkRangeFilter;
+
+        File file = constructFilename(metadata.keyspace, metadata.name, ssTable.getDataFileName());
+        Descriptor descriptor = Descriptor.fromFilename(file);
+        this.version = descriptor.version;
+
+        SummaryDbUtils.Summary summary = null;
+        Pair<DecoratedKey, DecoratedKey> keys = Pair.create(null, null);
+        try
+        {
+            now = System.nanoTime();
+            summary = SSTableCache.INSTANCE.keysFromSummary(metadata, ssTable);
+            stats.readSummaryDb(ssTable, System.nanoTime() - now);
+            keys = Pair.create(summary.first(), summary.last());
+        }
+        catch (IOException exception)
+        {
+            LOGGER.warn("Failed to read Summary.db file ssTable='{}'", ssTable, exception);
+        }
+
+        if (keys.left == null || keys.right == null)
+        {
+            LOGGER.warn("Could not load first and last key from Summary.db file, so attempting Index.db fileName={}",
+                        ssTable.getDataFileName());
+            now = System.nanoTime();
+            keys = SSTableCache.INSTANCE.keysFromIndex(metadata, ssTable);
+            stats.readIndexDb(ssTable, System.nanoTime() - now);
+        }
+
+        if (keys.left == null || keys.right == null)
+        {
+            throw new IOException("Could not load SSTable first or last tokens");
+        }
+
+        this.first = keys.left;
+        this.last = keys.right;
+        this.firstToken = ReaderUtils.tokenToBigInteger(first.getToken());
+        this.lastToken = ReaderUtils.tokenToBigInteger(last.getToken());
+        TokenRange readerRange = range();
+
+        List<PartitionKeyFilter> matchingKeyFilters = partitionKeyFilters.stream()
+                .filter(filter -> readerRange.contains(filter.token()))
+                .collect(Collectors.toList());
+        boolean overlapsSparkRange = sparkRangeFilter == null || SparkSSTableReader.overlaps(this, sparkRangeFilter.tokenRange());
+        if (!overlapsSparkRange  // SSTable doesn't overlap with Spark worker token range
+                || (matchingKeyFilters.isEmpty() && !partitionKeyFilters.isEmpty()))  // No matching partition key filters overlap with SSTable
+        {
+            this.partitionKeyFilters = Collections.emptyList();
+            stats.skippedSSTable(sparkRangeFilter, partitionKeyFilters, firstToken, lastToken);
+            LOGGER.info("Ignoring SSTableReader with firstToken={} lastToken={}, does not overlap with any filter",
+                        firstToken, lastToken);
+            statsMetadata = null;
+            header = null;
+            helper = null;
+            this.metadata = null;
+            return;
+        }
+
+        if (!matchingKeyFilters.isEmpty())
+        {
+            List<PartitionKeyFilter> matchInBloomFilter =
+                    ReaderUtils.filterKeyInBloomFilter(ssTable, metadata.partitioner, descriptor, matchingKeyFilters);
+            this.partitionKeyFilters = ImmutableList.copyOf(matchInBloomFilter);
+
+            // Check if required keys are actually present
+            if (matchInBloomFilter.isEmpty() || !ReaderUtils.anyFilterKeyInIndex(ssTable, matchInBloomFilter))
+            {
+                if (matchInBloomFilter.isEmpty())
+                {
+                    stats.missingInBloomFilter();
+                }
+                else
+                {
+                    stats.missingInIndex();
+                }
+                LOGGER.info("Ignoring SSTable {}, no match found in index file for key filters",
+                            this.ssTable.getDataFileName());
+                statsMetadata = null;
+                header = null;
+                helper = null;
+                this.metadata = null;
+                return;
+            }
+        }
+        else
+        {
+            this.partitionKeyFilters = ImmutableList.copyOf(partitionKeyFilters);
+        }
+
+        Map<MetadataType, MetadataComponent> componentMap = SSTableCache.INSTANCE.componentMapFromStats(ssTable, descriptor);
+
+        ValidationMetadata validation = (ValidationMetadata) componentMap.get(MetadataType.VALIDATION);
+        if (validation != null && !validation.partitioner.equals(metadata.partitioner.getClass().getName()))
+        {
+            throw new IllegalStateException("Partitioner in ValidationMetadata does not match TableMetaData: "
+                                          + validation.partitioner + " vs. " + metadata.partitioner.getClass().getName());
+        }
+
+        this.statsMetadata = (StatsMetadata) componentMap.get(MetadataType.STATS);
+        SerializationHeader.Component headerComp = (SerializationHeader.Component) componentMap.get(MetadataType.HEADER);
+        if (headerComp == null)
+        {
+            throw new IOException("Cannot read SSTable if cannot deserialize stats header info");
+        }
+
+        if (useIncrementalRepair && !isRepairPrimary && isRepaired())
+        {
+            stats.skippedRepairedSSTable(ssTable, statsMetadata.repairedAt);
+            LOGGER.info("Ignoring repaired SSTable on non-primary repair replica ssTable='{}' repairedAt={}",
+                        ssTable, statsMetadata.repairedAt);
+            header = null;
+            helper = null;
+            this.metadata = null;
+            return;
+        }
+
+        Set<String> columnNames = Streams.concat(metadata.columns().stream(),
+                                                 metadata.staticColumns().stream())
+                                         .map(column -> column.name.toString())
+                                         .collect(Collectors.toSet());
+        Map<ByteBuffer, DroppedColumn> droppedColumns = new HashMap<>();
+        droppedColumns.putAll(buildDroppedColumns(metadata.keyspace,
+                                                  metadata.name,
+                                                  ssTable,
+                                                  headerComp.getRegularColumns(),
+                                                  columnNames,
+                                                  ColumnMetadata.Kind.REGULAR));
+        droppedColumns.putAll(buildDroppedColumns(metadata.keyspace,
+                                                  metadata.name,
+                                                  ssTable,
+                                                  headerComp.getStaticColumns(),
+                                                  columnNames,
+                                                  ColumnMetadata.Kind.STATIC));
+        if (!droppedColumns.isEmpty())
+        {
+            LOGGER.info("Rebuilding table metadata with dropped columns numDroppedColumns={} ssTable='{}'",
+                        droppedColumns.size(), ssTable);
+            metadata = metadata.unbuild().droppedColumns(droppedColumns).build();
+        }
+
+        this.header = headerComp.toHeader(metadata);
+        this.helper = new DeserializationHelper(metadata,
+                                                MessagingService.VERSION_30,
+                                                DeserializationHelper.Flag.FROM_REMOTE,
+                                                buildColumnFilter(metadata, columnFilter));
+        this.metadata = metadata;
+
+        if (readIndexOffset && summary != null)
+        {
+            SummaryDbUtils.Summary finalSummary = summary;
+            extractRange(sparkRangeFilter, partitionKeyFilters)
+                    .ifPresent(range -> readOffsets(finalSummary.summary(), range));
+        }
+        else
+        {
+            LOGGER.warn("Reading SSTable without looking up start/end offset, performance will potentially be degraded");
+        }
+
+        // Open SSTableStreamReader so opened in parallel inside thread pool
+        // and buffered + ready to go when CompactionIterator starts reading
+        reader.set(new SSTableStreamReader());
+        stats.openedSSTable(ssTable, System.nanoTime() - startTimeNanos);
+        this.openedNanos = System.nanoTime();
+    }
+
+    /**
+     * Constructs full file path for a given combination of keyspace, table, and data file name,
+     * while adjusting for data files with non-standard names prefixed with keyspace and table
+     *
+     * @param keyspace Name of the keyspace
+     * @param table    Name of the table
+     * @param filename Name of the data file
+     * @return A full file path, adjusted for non-standard file names
+     */
+    @VisibleForTesting
+    @NotNull
+    static File constructFilename(@NotNull String keyspace, @NotNull String table, @NotNull String filename)
+    {
+        String[] components = filename.split("-");
+        if (components.length == 6
+                && components[0].equals(keyspace)
+                && components[1].equals(table))
+        {
+            filename = filename.substring(keyspace.length() + table.length() + 2);
+        }
+
+        return new File(String.format("./%s/%s", keyspace, table), filename);
+    }
+
+    private static Map<ByteBuffer, DroppedColumn> buildDroppedColumns(String keyspace,
+                                                                      String table,
+                                                                      SSTable ssTable,
+                                                                      Map<ByteBuffer, AbstractType<?>> columns,
+                                                                      Set<String> columnNames,
+                                                                      ColumnMetadata.Kind kind)
+    {
+        Map<ByteBuffer, DroppedColumn> droppedColumns = new HashMap<>();
+        for (Map.Entry<ByteBuffer, AbstractType<?>> entry : columns.entrySet())
+        {
+            String colName = UTF8Type.instance.getString((entry.getKey()));
+            if (!columnNames.contains(colName))
+            {
+                AbstractType<?> type = entry.getValue();
+                LOGGER.warn("Dropped column found colName={} sstable='{}'", colName, ssTable);
+                ColumnMetadata column = new ColumnMetadata(keyspace,
+                                                           table,
+                                                           ColumnIdentifier.getInterned(colName, true),
+                                                           type,
+                                                           ColumnMetadata.NO_POSITION,
+                                                           kind);
+                long droppedTime = TimeUnit.MILLISECONDS.toMicros(System.currentTimeMillis())
+                                 - TimeUnit.MINUTES.toMicros(60);
+                droppedColumns.put(entry.getKey(), new DroppedColumn(column, droppedTime));
+            }
+        }
+        return droppedColumns;
+    }
+
+    /**
+     * Merge all the partition key filters to give the token range we care about.
+     * If no partition key filters, then use the Spark worker token range.
+     *
+     * @param sparkRangeFilter    optional spark range filter
+     * @param partitionKeyFilters list of partition key filters
+     * @return the token range we care about for this Spark worker
+     */
+    public static Optional<TokenRange> extractRange(@Nullable SparkRangeFilter sparkRangeFilter,
+                                                    @NotNull List<PartitionKeyFilter> partitionKeyFilters)
+    {
+        Optional<TokenRange> partitionKeyRange = partitionKeyFilters.stream()
+                                                                    .map(PartitionKeyFilter::tokenRange)
+                                                                    .reduce(TokenRange::merge);
+        return partitionKeyRange.isPresent()
+                ? partitionKeyRange
+                : Optional.ofNullable(sparkRangeFilter != null ? sparkRangeFilter.tokenRange() : null);
+    }
+
+    /**
+     * Read Data.db offsets by binary searching Summary.db into Index.db, then reading offsets in Index.db
+     *
+     * @param indexSummary Summary.db index summary
+     * @param range        token range we care about for this Spark worker
+     */
+    private void readOffsets(IndexSummary indexSummary, TokenRange range)
+    {
+        try
+        {
+            // If start is null we failed to find an overlapping token in the Index.db file,
+            // this is unlikely as we already pre-filter the SSTable based on the start-end token range.
+            // But in this situation we read the entire Data.db file to be safe, even if it hits performance.
+            startOffset = IndexDbUtils.findDataDbOffset(indexSummary, range, metadata.partitioner, ssTable, stats);
+            if (startOffset == null)
+            {
+                LOGGER.error("Failed to find Data.db start offset, performance will be degraded sstable='{}'", ssTable);
+            }
+        }
+        catch (IOException exception)
+        {
+            LOGGER.warn("IOException finding SSTable offsets, cannot skip directly to start offset in Data.db. "
+                      + "Performance will be degraded.", exception);
+        }
+    }
+
+    /**
+     * Build a ColumnFilter if we need to prune any columns for more efficient deserialization of the SSTable
+     *
+     * @param metadata     TableMetadata object
+     * @param columnFilter prune column filter
+     * @return ColumnFilter if and only if we can prune any columns when deserializing the SSTable,
+     *                      otherwise return null
+     */
+    @Nullable
+    private static ColumnFilter buildColumnFilter(TableMetadata metadata, @Nullable PruneColumnFilter columnFilter)
+    {
+        if (columnFilter == null)
+        {
+            return null;
+        }
+        List<ColumnMetadata> include = metadata.columns().stream()
+                .filter(column -> columnFilter.includeColumn(column.name.toString()))
+                .collect(Collectors.toList());
+        if (include.size() == metadata.columns().size())
+        {
+            return null;  // No columns pruned
+        }
+        return ColumnFilter.allRegularColumnsBuilder(metadata, false)
+                           .addAll(include)
+                           .build();
+    }
+
+    public SSTable sstable()
+    {
+        return ssTable;
+    }
+
+    public boolean ignore()
+    {
+        return reader.get() == null;
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Objects.hash(metadata.keyspace, metadata.name, ssTable);
+    }
+
+    @Override
+    public boolean equals(Object other)
+    {
+        return other instanceof SSTableReader
+            && this.metadata.keyspace.equals(((SSTableReader) other).metadata.keyspace)
+            && this.metadata.name.equals(((SSTableReader) other).metadata.name)
+            && this.ssTable.equals(((SSTableReader) other).ssTable);
+    }
+
+    public boolean isRepaired()
+    {
+        return isRepaired.apply(statsMetadata);
+    }
+
+    public DecoratedKey first()
+    {
+        return first;
+    }
+
+    public DecoratedKey last()
+    {
+        return last;
+    }
+
+    public long getMinTimestamp()
+    {
+        return statsMetadata.minTimestamp;
+    }
+
+    public long getMaxTimestamp()
+    {
+        return statsMetadata.maxTimestamp;
+    }
+
+    public StatsMetadata getSSTableMetadata()
+    {
+        return statsMetadata;
+    }
+
+    @Override
+    public ISSTableScanner scanner()
+    {
+        ISSTableScanner result = reader.getAndSet(null);
+        if (result == null)
+        {
+            throw new IllegalStateException("SSTableStreamReader cannot be re-used");
+        }
+        return result;
+    }
+
+    @Override
+    @NotNull
+    public BigInteger firstToken()
+    {
+        return firstToken;
+    }
+
+    @Override
+    @NotNull
+    public BigInteger lastToken()
+    {
+        return lastToken;
+    }
+
+    public class SSTableStreamReader implements ISSTableScanner
+    {
+        private final DataInputStream dis;
+        private final DataInputPlus in;
+        final RawInputStream dataStream;
+        private DecoratedKey key;
+        private DeletionTime partitionLevelDeletion;
+        private SSTableSimpleIterator iterator;
+        private Row staticRow;
+        @Nullable
+        private final BigInteger lastToken;
+        private long lastTimeNanos = System.nanoTime();
+
+        SSTableStreamReader() throws IOException
+        {
+            lastToken = sparkRangeFilter != null ? sparkRangeFilter.tokenRange().upperEndpoint() : null;
+            try (@Nullable InputStream compressionInfoInputStream = ssTable.openCompressionStream())
+            {
+                DataInputStream dataInputStream = new DataInputStream(ssTable.openDataStream());
+
+                if (compressionInfoInputStream != null)
+                {
+                    dataStream = CompressedRawInputStream.fromInputStream(ssTable,
+                                                                          dataInputStream,
+                                                                          compressionInfoInputStream,
+                                                                          version.hasMaxCompressedLength(),
+                                                                          stats);
+                }
+                else
+                {
+                    dataStream = new RawInputStream(dataInputStream, new byte[64 * 1024], stats);
+                }
+            }
+            dis = new DataInputStream(dataStream);
+            if (startOffset != null)
+            {
+                // Skip to start offset, if known, of first in-range partition
+                ByteBufferUtils.skipFully(dis, startOffset);
+                assert dataStream.position() == startOffset;
+                LOGGER.info("Using Data.db start offset to skip ahead startOffset={} sstable='{}'",
+                            startOffset, ssTable);
+                stats.skippedDataDbStartOffset(startOffset);
+            }
+            in = new DataInputPlus.DataInputStreamPlus(dis);
+        }
+
+        @Override
+        public TableMetadata metadata()
+        {
+            return metadata;
+        }
+
+        public boolean overlapsSparkTokenRange(BigInteger token)
+        {
+            return sparkRangeFilter == null || sparkRangeFilter.overlaps(token);
+        }
+
+        public boolean overlapsPartitionFilters(DecoratedKey key)
+        {
+            return partitionKeyFilters.isEmpty()
+                || partitionKeyFilters.stream().anyMatch(filter -> filter.matches(key.getKey()));
+        }
+
+        public boolean overlaps(DecoratedKey key, BigInteger token)
+        {
+            return overlapsSparkTokenRange(token) && overlapsPartitionFilters(key);
+        }
+
+        @Override
+        public boolean hasNext()
+        {
+            try
+            {
+                while (true)
+                {
+                    key = metadata.partitioner.decorateKey(ByteBufferUtil.readWithShortLength(in));
+                    partitionLevelDeletion = DeletionTime.serializer.deserialize(in);
+                    iterator = SSTableSimpleIterator.create(metadata, in, header, helper, partitionLevelDeletion);
+                    staticRow = iterator.readStaticRow();
+                    BigInteger token = ReaderUtils.tokenToBigInteger(key.getToken());
+                    if (overlaps(key, token))
+                    {
+                        // Partition overlaps with filters
+                        long now = System.nanoTime();
+                        stats.nextPartition(now - lastTimeNanos);
+                        lastTimeNanos = now;
+                        return true;
+                    }
+                    if (lastToken != null && startOffset != null && lastToken.compareTo(token) < 0)
+                    {
+                        // Partition no longer overlaps SparkTokenRange so we've finished reading this SSTable
+                        stats.skippedDataDbEndOffset(dataStream.position() - startOffset);
+                        return false;
+                    }
+                    stats.skippedPartition(key.getKey(), ReaderUtils.tokenToBigInteger(key.getToken()));
+                    // Skip partition efficiently without deserializing
+                    UnfilteredDeserializer deserializer = UnfilteredDeserializer.create(metadata, in, header, helper);
+                    while (deserializer.hasNext())
+                    {
+                        deserializer.skipNext();
+                    }
+                }
+            }
+            catch (EOFException exception)
+            {
+                return false;
+            }
+            catch (IOException exception)
+            {
+                stats.corruptSSTable(exception, metadata.keyspace, metadata.name, ssTable);
+                LOGGER.warn("IOException reading sstable keyspace={} table={} dataFileName={} ssTable='{}'",
+                            metadata.keyspace, metadata.name, ssTable.getDataFileName(), ssTable, exception);
+                throw new SSTableStreamException(exception);
+            }
+            catch (Throwable throwable)
+            {
+                stats.corruptSSTable(throwable, metadata.keyspace, metadata.name, ssTable);
+                LOGGER.error("Error reading sstable keyspace={} table={}  dataFileName={} ssTable='{}'",
+                             metadata.keyspace, metadata.name, ssTable.getDataFileName(), ssTable, throwable);
+                throw new RuntimeException(ThrowableUtils.rootCause(throwable));
+            }
+        }
+
+        @Override
+        public UnfilteredRowIterator next()
+        {
+            return new UnfilteredIterator();
+        }
+
+        @Override
+        public void close()
+        {
+            LOGGER.debug("Closing SparkSSTableReader {}", ssTable);
+            try
+            {
+                dis.close();
+                if (openedNanos != null)
+                {
+                    stats.closedSSTable(System.nanoTime() - openedNanos);
+                }
+            }
+            catch (IOException exception)
+            {
+                LOGGER.warn("IOException closing SSTable DataInputStream", exception);
+            }
+        }
+
+        @Override
+        public long getLengthInBytes()
+        {
+            // This is mostly used to return Compaction info for Metrics or via JMX so we can ignore here
+            return 0;
+        }
+
+        @Override
+        public long getCompressedLengthInBytes()
+        {
+            return 0;
+        }
+
+        @Override
+        public long getCurrentPosition()
+        {
+            // This is mostly used to return Compaction info for Metrics or via JMX so we can ignore here
+            return 0;
+        }
+
+        @Override
+        public long getBytesScanned()
+        {
+            return 0;
+        }
+
+        @Override
+        public Set<org.apache.cassandra.io.sstable.format.SSTableReader> getBackingSSTables()
+        {
+            return Collections.emptySet();
+        }
+
+        private class UnfilteredIterator implements UnfilteredRowIterator
+        {
+            @Override
+            public RegularAndStaticColumns columns()
+            {
+                return metadata.regularAndStaticColumns();
+            }
+
+            @Override
+            public TableMetadata metadata()
+            {
+                return metadata;
+            }
+
+            @Override
+            public boolean isReverseOrder()
+            {
+                return false;
+            }
+
+            @Override
+            public DecoratedKey partitionKey()
+            {
+                return key;
+            }
+
+            @Override
+            public DeletionTime partitionLevelDeletion()
+            {
+                return partitionLevelDeletion;
+            }
+
+            @Override
+            public Row staticRow()
+            {
+                return staticRow;
+            }
+
+            @Override
+            public EncodingStats stats()
+            {
+                return header.stats();
+            }
+
+            @Override
+            public boolean hasNext()
+            {
+                try
+                {
+                    return iterator.hasNext();
+                }
+                catch (IOError error)
+                {
+                    // SSTableSimpleIterator::computeNext wraps IOException in IOError, so we catch those,
+                    // try to extract the IOException and re-wrap it in an SSTableStreamException,
+                    // which we can then process in TableStreamScanner
+                    if (error.getCause() instanceof IOException)
+                    {
+                        throw new SSTableStreamException((IOException) error.getCause());
+                    }
+
+                    // Otherwise, just throw the IOError and deal with it further up the stack
+                    throw error;
+                }
+            }
+
+            @Override
+            public Unfiltered next()
+            {
+                // NOTE: In practice we know that IOException will be thrown by hasNext(),
+                //       because that's where the actual reading happens, so we don't bother
+                //       catching IOError here (contrarily to what we do in hasNext)
+                return iterator.next();
+            }
+
+            @Override
+            public void close()
+            {
+            }
+        }
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/reader/Scannable.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/reader/Scannable.java
new file mode 100644
index 0000000..73f7856
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/reader/Scannable.java
@@ -0,0 +1,27 @@
+/*
+ * 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.cassandra.spark.reader;
+
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+
+public interface Scannable
+{
+    ISSTableScanner scanner();
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/reader/SchemaBuilder.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/reader/SchemaBuilder.java
new file mode 100644
index 0000000..c09eae8
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/reader/SchemaBuilder.java
@@ -0,0 +1,547 @@
+/*
+ * 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.cassandra.spark.reader;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.antlr.runtime.RecognitionException;
+import org.apache.cassandra.bridge.CassandraBridge;
+import org.apache.cassandra.bridge.CassandraBridgeImplementation;
+import org.apache.cassandra.bridge.CassandraSchema;
+import org.apache.cassandra.cql3.CQL3Type;
+import org.apache.cassandra.cql3.CQLFragmentParser;
+import org.apache.cassandra.cql3.CqlParser;
+import org.apache.cassandra.cql3.statements.schema.CreateTypeStatement;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.CollectionType;
+import org.apache.cassandra.db.marshal.ListType;
+import org.apache.cassandra.db.marshal.MapType;
+import org.apache.cassandra.db.marshal.SetType;
+import org.apache.cassandra.db.marshal.TupleType;
+import org.apache.cassandra.db.marshal.UserType;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.KeyspaceMetadata;
+import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableId;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.schema.Types;
+import org.apache.cassandra.spark.data.CqlField;
+import org.apache.cassandra.spark.data.CqlTable;
+import org.apache.cassandra.spark.data.ReplicationFactor;
+import org.apache.cassandra.spark.data.complex.CqlFrozen;
+import org.apache.cassandra.spark.data.complex.CqlUdt;
+import org.apache.cassandra.spark.data.partitioner.Partitioner;
+import org.apache.cassandra.utils.Pair;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+public class SchemaBuilder
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(SchemaBuilder.class);
+
+    private final TableMetadata metadata;
+    private final KeyspaceMetadata keyspaceMetadata;
+    private final String createStmt;
+    private final String keyspace;
+    private final ReplicationFactor replicationFactor;
+    private final CassandraBridge bridge;
+    private final int indexCount;
+
+    public SchemaBuilder(CqlTable table, Partitioner partitioner)
+    {
+        this(table, partitioner, null);
+    }
+
+    public SchemaBuilder(CqlTable table, Partitioner partitioner, UUID tableId)
+    {
+        this(table.createStatement(),
+             table.keyspace(),
+             table.replicationFactor(),
+             partitioner,
+             table.udtCreateStmts(),
+             tableId,
+             0);
+    }
+
+    @VisibleForTesting
+    public SchemaBuilder(String createStmt, String keyspace, ReplicationFactor replicationFactor)
+    {
+        this(createStmt, keyspace, replicationFactor, Partitioner.Murmur3Partitioner, Collections.emptySet(), null, 0);
+    }
+
+    @VisibleForTesting
+    public SchemaBuilder(String createStmt,
+                         String keyspace,
+                         ReplicationFactor replicationFactor,
+                         Partitioner partitioner)
+    {
+        this(createStmt, keyspace, replicationFactor, partitioner, Collections.emptySet(), null, 0);
+    }
+
+    public SchemaBuilder(String createStmt,
+                         String keyspace,
+                         ReplicationFactor replicationFactor,
+                         Partitioner partitioner,
+                         Set<String> udtStmts,
+                         @Nullable UUID tableId,
+                         int indexCount)
+    {
+        this.createStmt = createStmt;
+        this.keyspace = keyspace;
+        this.replicationFactor = replicationFactor;
+        this.bridge = new CassandraBridgeImplementation();
+        this.indexCount = indexCount;
+
+        Pair<KeyspaceMetadata, TableMetadata> updated = CassandraSchema.apply(schema ->
+                updateSchema(schema,
+                             this.keyspace,
+                             udtStmts,
+                             this.createStmt,
+                             partitioner,
+                             this.replicationFactor,
+                             tableId,
+                             this::validateColumnMetaData));
+        this.keyspaceMetadata = updated.left;
+        this.metadata = updated.right;
+    }
+
+    // Update schema with the given keyspace, table and udt.
+    // It creates the cooresponding metadata and opens instances for keyspace and table, if needed.
+    // At the end, it validates that the input keyspace and table both should have metadata exist and instance opened.
+    private static Pair<KeyspaceMetadata, TableMetadata> updateSchema(Schema schema,
+                                                                      String keyspace,
+                                                                      Set<String> udtStatements,
+                                                                      String createStatement,
+                                                                      Partitioner partitioner,
+                                                                      ReplicationFactor replicationFactor,
+                                                                      UUID tableId,
+                                                                      Consumer<ColumnMetadata> columnValidator)
+    {
+        // Set up and open keyspace if needed
+        IPartitioner cassPartitioner = CassandraBridgeImplementation.getPartitioner(partitioner);
+        setupKeyspace(schema, keyspace, replicationFactor, cassPartitioner);
+
+        // Set up and open table if needed, parse UDTs and include when parsing table schema
+        List<CreateTypeStatement.Raw> typeStatements = new ArrayList<>(udtStatements.size());
+        for (String udt : udtStatements)
+        {
+            try
+            {
+                typeStatements.add((CreateTypeStatement.Raw) CQLFragmentParser
+                        .parseAnyUnhandled(CqlParser::query, udt));
+            }
+            catch (RecognitionException exception)
+            {
+                LOGGER.error("Failed to parse type expression '{}'", udt);
+                throw new IllegalStateException(exception);
+            }
+        }
+        Types.RawBuilder typesBuilder = Types.rawBuilder(keyspace);
+        for (CreateTypeStatement.Raw st : typeStatements)
+        {
+            st.addToRawBuilder(typesBuilder);
+        }
+        Types types = typesBuilder.build();
+        TableMetadata.Builder builder = CQLFragmentParser
+                .parseAny(CqlParser::createTableStatement, createStatement, "CREATE TABLE")
+                .keyspace(keyspace)
+                .prepare(null)
+                .builder(types)
+                .partitioner(cassPartitioner);
+
+        if (tableId != null)
+        {
+            builder.id(TableId.fromUUID(tableId));
+        }
+        TableMetadata tableMetadata = builder.build();
+        tableMetadata.columns().forEach(columnValidator);
+        setupTableAndUdt(schema, keyspace, tableMetadata, types);
+
+        return validateKeyspaceTable(schema, keyspace, tableMetadata.name);
+    }
+
+    private void validateColumnMetaData(@NotNull ColumnMetadata column)
+    {
+        validateType(column.type);
+    }
+
+    private void validateType(AbstractType<?> type)
+    {
+        validateType(type.asCQL3Type());
+    }
+
+    private void validateType(CQL3Type cqlType)
+    {
+        if (!(cqlType instanceof CQL3Type.Native)
+                && !(cqlType instanceof CQL3Type.Collection)
+                && !(cqlType instanceof CQL3Type.UserDefined)
+                && !(cqlType instanceof CQL3Type.Tuple))
+        {
+            throw new UnsupportedOperationException("Only native, collection, tuples or UDT data types are supported, "
+                                                  + "unsupported data type: " + cqlType.toString());
+        }
+
+        if (cqlType instanceof CQL3Type.Native)
+        {
+            CqlField.CqlType type = bridge.parseType(cqlType.toString());
+            if (!type.isSupported())
+            {
+                throw new UnsupportedOperationException(type.name() + " data type is not supported");
+            }
+        }
+        else if (cqlType instanceof CQL3Type.Collection)
+        {
+            // Validate collection inner types
+            CQL3Type.Collection collection = (CQL3Type.Collection) cqlType;
+            CollectionType<?> type = (CollectionType<?>) collection.getType();
+            switch (type.kind)
+            {
+                case LIST:
+                    validateType(((ListType<?>) type).getElementsType());
+                    return;
+                case SET:
+                    validateType(((SetType<?>) type).getElementsType());
+                    return;
+                case MAP:
+                    validateType(((MapType<?, ?>) type).getKeysType());
+                    validateType(((MapType<?, ?>) type).getValuesType());
+                    return;
+                default:
+                    // Do nothing
+            }
+        }
+        else if (cqlType instanceof CQL3Type.Tuple)
+        {
+            CQL3Type.Tuple tuple = (CQL3Type.Tuple) cqlType;
+            TupleType tupleType = (TupleType) tuple.getType();
+            for (AbstractType<?> subType : tupleType.allTypes())
+            {
+                validateType(subType);
+            }
+        }
+        else
+        {
+            // Validate UDT inner types
+            UserType userType = (UserType) ((CQL3Type.UserDefined) cqlType).getType();
+            for (AbstractType<?> innerType : userType.fieldTypes())
+            {
+                validateType(innerType);
+            }
+        }
+    }
+
+    private static boolean keyspaceMetadataExists(Schema schema, String keyspaceName)
+    {
+        return schema.getKeyspaceMetadata(keyspaceName) != null;
+    }
+
+    private static boolean tableMetadataExists(Schema schema, String keyspaceName, String tableName)
+    {
+        KeyspaceMetadata ksMetadata = schema.getKeyspaceMetadata(keyspaceName);
+        if (ksMetadata == null)
+        {
+            return false;
+        }
+
+        return ksMetadata.hasTable(tableName);
+    }
+
+    private static boolean keyspaceInstanceExists(Schema schema, String keyspaceName)
+    {
+        return schema.getKeyspaceInstance(keyspaceName) != null;
+    }
+
+    private static boolean tableInstanceExists(Schema schema, String keyspaceName, String tableName)
+    {
+        Keyspace keyspace = schema.getKeyspaceInstance(keyspaceName);
+        if (keyspace == null)
+        {
+            return false;
+        }
+
+        try
+        {
+            keyspace.getColumnFamilyStore(tableName);
+        }
+        catch (IllegalArgumentException exception)
+        {
+            LOGGER.info("Table instance does not exist. keyspace={} table={} existingCFS={}",
+                        keyspace, tableName, keyspace.getColumnFamilyStores());
+            return false;
+        }
+        return true;
+    }
+
+    // Check whether keyspace metadata exists. Create keyspace metadata, if not.
+    // Check whether keyspace instance is opened. Open the keyspace, if not.
+    // NOTE: It is possible that external code that just creates metadata, but does not open the keyspace
+    private static void setupKeyspace(Schema schema,
+                                      String keyspaceName,
+                                      ReplicationFactor replicationFactor,
+                                      IPartitioner partitioner)
+    {
+        if (!keyspaceMetadataExists(schema, keyspaceName))
+        {
+            LOGGER.info("Setting up keyspace metadata in schema keyspace={} rfStrategy={} partitioner={}",
+                        keyspaceName, replicationFactor.getReplicationStrategy().name(), partitioner);
+            KeyspaceMetadata keyspaceMetadata =
+                    KeyspaceMetadata.create(keyspaceName, KeyspaceParams.create(true, rfToMap(replicationFactor)));
+            schema.load(keyspaceMetadata);
+        }
+
+        if (!keyspaceInstanceExists(schema, keyspaceName))
+        {
+            LOGGER.info("Setting up keyspace instance in schema keyspace={} rfStrategy={} partitioner={}",
+                        keyspaceName, replicationFactor.getReplicationStrategy().name(), partitioner);
+            // Create keyspace instance and also initCf (cfs) for the table
+            Keyspace.openWithoutSSTables(keyspaceName);
+        }
+    }
+
+    // Check whether table metadata exists. Create table metadata, if not.
+    // Check whether table instance is opened. Open/init the table, if not.
+    // NOTE: It is possible that external code that just creates metadata, but does not open the table
+    private static void setupTableAndUdt(Schema schema,
+                                         String keyspaceName,
+                                         TableMetadata tableMetadata,
+                                         Types userTypes)
+    {
+        String tableName = tableMetadata.name;
+        KeyspaceMetadata keyspaceMetadata = schema.getKeyspaceMetadata(keyspaceName);
+        if (keyspaceMetadata == null)
+        {
+            LOGGER.error("Keyspace metadata does not exist. keyspace={}", keyspaceName);
+            throw new IllegalStateException("Keyspace metadata null for '" + keyspaceName
+                                          + "' when it should have been initialized already");
+        }
+
+        if (!tableMetadataExists(schema, keyspaceName, tableName))
+        {
+            LOGGER.info("Setting up table metadata in schema keyspace={} table={} partitioner={}",
+                        keyspaceName, tableName, tableMetadata.partitioner.getClass().getName());
+            keyspaceMetadata = keyspaceMetadata.withSwapped(keyspaceMetadata.tables.with(tableMetadata));
+            schema.load(keyspaceMetadata);
+        }
+
+        // The metadata of the table might not be the input tableMetadata. Fetch the current to be safe.
+        TableMetadata currentTable = schema.getTableMetadata(keyspaceName, tableName);
+        if (!tableInstanceExists(schema, keyspaceName, tableName))
+        {
+            LOGGER.info("Setting up table instance in schema keyspace={} table={} partitioner={}",
+                        keyspaceName, tableName, tableMetadata.partitioner.getClass().getName());
+            if (keyspaceInstanceExists(schema, keyspaceName))
+            {
+                // initCf (cfs) in the opened keyspace
+                schema.getKeyspaceInstance(keyspaceName)
+                      .initCf(TableMetadataRef.forOfflineTools(currentTable), false);
+            }
+            else
+            {
+                // The keyspace has not yet opened, create/open keyspace instance and also initCf (cfs) for the table
+                Keyspace.openWithoutSSTables(keyspaceName);
+            }
+        }
+
+        if (!userTypes.equals(Types.none()))
+        {
+            LOGGER.info("Setting up user types in schema keyspace={} types={}",
+                        keyspaceName, userTypes);
+            // Update Schema instance with any user-defined types built
+            keyspaceMetadata = keyspaceMetadata.withSwapped(userTypes);
+            schema.load(keyspaceMetadata);
+        }
+    }
+
+    private static Pair<KeyspaceMetadata, TableMetadata> validateKeyspaceTable(Schema schema,
+                                                                               String keyspaceName,
+                                                                               String tableName)
+    {
+        Preconditions.checkState(keyspaceMetadataExists(schema, keyspaceName),
+                                 "Keyspace metadata does not exist after building schema. keyspace=%s",
+                                 keyspaceName);
+        Preconditions.checkState(keyspaceInstanceExists(schema, keyspaceName),
+                                 "Keyspace instance is not opened after building schema. keyspace=%s",
+                                 keyspaceName);
+        Preconditions.checkState(tableMetadataExists(schema, keyspaceName, tableName),
+                                 "Table metadata does not exist after building schema. keyspace=%s table=%s",
+                                 keyspaceName, tableName);
+        Preconditions.checkState(tableInstanceExists(schema, keyspaceName, tableName),
+                                 "Table instance is not opened after building schema. keyspace=%s table=%s",
+                                 keyspaceName, tableName);
+
+        // Validated above that keyspace and table, both exist and are opened
+        KeyspaceMetadata keyspaceMetadata = schema.getKeyspaceMetadata(keyspaceName);
+        TableMetadata tableMetadata = schema.getTableMetadata(keyspaceName, tableName);
+        return Pair.create(keyspaceMetadata, tableMetadata);
+    }
+
+    public TableMetadata tableMetaData()
+    {
+        return metadata;
+    }
+
+    public String createStatement()
+    {
+        return createStmt;
+    }
+
+    public CqlTable build()
+    {
+        Map<String, CqlField.CqlUdt> udts = buildsUdts(keyspaceMetadata);
+        List<CqlField> fields = buildFields(metadata, udts).stream().sorted().collect(Collectors.toList());
+        return new CqlTable(keyspace,
+                            metadata.name,
+                            createStmt,
+                            replicationFactor,
+                            fields,
+                            new HashSet<>(udts.values()),
+                            indexCount);
+    }
+
+    private Map<String, CqlField.CqlUdt> buildsUdts(KeyspaceMetadata keyspaceMetadata)
+    {
+        List<UserType> userTypes = new ArrayList<>();
+        keyspaceMetadata.types.forEach(userTypes::add);
+        Map<String, CqlField.CqlUdt> udts = new HashMap<>(userTypes.size());
+        while (!userTypes.isEmpty())
+        {
+            UserType userType = userTypes.remove(0);
+            if (!SchemaBuilder.nestedUdts(userType).stream().allMatch(udts::containsKey))
+            {
+                // This UDT contains a nested user-defined type that has not been parsed yet
+                // so re-add to the queue and parse later
+                userTypes.add(userType);
+                continue;
+            }
+            String name = userType.getNameAsString();
+            CqlUdt.Builder builder = CqlUdt.builder(keyspaceMetadata.name, name);
+            for (int field = 0; field < userType.size(); field++)
+            {
+                builder.withField(userType.fieldName(field).toString(),
+                                  bridge.parseType(userType.fieldType(field).asCQL3Type().toString(), udts));
+            }
+            udts.put(name, builder.build());
+        }
+
+        return udts;
+    }
+
+    /**
+     * @param type an abstract type
+     * @return a set of UDTs nested within the type parameter
+     */
+    private static Set<String> nestedUdts(AbstractType<?> type)
+    {
+        Set<String> result = new HashSet<>();
+        nestedUdts(type, result, false);
+        return result;
+    }
+
+    private static void nestedUdts(AbstractType<?> type, Set<String> udts, boolean isNested)
+    {
+        if (type instanceof UserType)
+        {
+            if (isNested)
+            {
+                udts.add(((UserType) type).getNameAsString());
+            }
+            for (AbstractType<?> nestedType : ((UserType) type).fieldTypes())
+            {
+                nestedUdts(nestedType, udts, true);
+            }
+        }
+        else if (type instanceof TupleType)
+        {
+            for (AbstractType<?> nestedType : ((TupleType) type).allTypes())
+            {
+                nestedUdts(nestedType, udts, true);
+            }
+        }
+        else if (type instanceof SetType)
+        {
+            nestedUdts(((SetType<?>) type).getElementsType(), udts, true);
+        }
+        else if (type instanceof ListType)
+        {
+            nestedUdts(((ListType<?>) type).getElementsType(), udts, true);
+        }
+        else if (type instanceof MapType)
+        {
+            nestedUdts(((MapType<?, ?>) type).getKeysType(), udts, true);
+            nestedUdts(((MapType<?, ?>) type).getValuesType(), udts, true);
+        }
+    }
+
+    private List<CqlField> buildFields(TableMetadata metadata, Map<String, CqlField.CqlUdt> udts)
+    {
+        Iterator<ColumnMetadata> it = metadata.allColumnsInSelectOrder();
+        List<CqlField> result = new ArrayList<>();
+        int position = 0;
+        while (it.hasNext())
+        {
+            ColumnMetadata col = it.next();
+            boolean isPartitionKey = col.isPartitionKey();
+            boolean isClusteringColumn = col.isClusteringColumn();
+            boolean isStatic = col.isStatic();
+            String name = col.name.toCQLString();
+            CqlField.CqlType type = col.type.isUDT() ? udts.get(((UserType) col.type).getNameAsString())
+                                                     : bridge.parseType(col.type.asCQL3Type().toString(), udts);
+            boolean isFrozen = col.type.isFreezable() && !col.type.isMultiCell();
+            result.add(new CqlField(isPartitionKey,
+                                    isClusteringColumn,
+                                    isStatic,
+                                    name,
+                                    !(type instanceof CqlFrozen) && isFrozen ? CqlFrozen.build(type) : type,
+                                    position));
+            position++;
+        }
+        return result;
+    }
+
+    static Map<String, String> rfToMap(ReplicationFactor replicationFactor)
+    {
+        Map<String, String> result = new HashMap<>(replicationFactor.getOptions().size() + 1);
+        result.put("class", "org.apache.cassandra.locator." + replicationFactor.getReplicationStrategy().name());
+        for (Map.Entry<String, Integer> entry : replicationFactor.getOptions().entrySet())
+        {
+            result.put(entry.getKey(), Integer.toString(entry.getValue()));
+        }
+        return result;
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/reader/SummaryDbUtils.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/reader/SummaryDbUtils.java
new file mode 100644
index 0000000..a667dcc
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/reader/SummaryDbUtils.java
@@ -0,0 +1,201 @@
+/*
+ * 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.cassandra.spark.reader;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.sstable.IndexSummary;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.spark.data.SSTable;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Helper methods for reading the Summary.db SSTable file component
+ */
+final class SummaryDbUtils
+{
+    static class Summary
+    {
+        private final IndexSummary indexSummary;
+        private final DecoratedKey firstKey;
+        private final DecoratedKey lastKey;
+
+        Summary(IndexSummary indexSummary,
+                DecoratedKey firstKey,
+                DecoratedKey lastKey)
+        {
+            this.indexSummary = indexSummary;
+            this.firstKey = firstKey;
+            this.lastKey = lastKey;
+        }
+
+        public IndexSummary summary()
+        {
+            return indexSummary;
+        }
+
+        public DecoratedKey first()
+        {
+            return firstKey;
+        }
+
+        public DecoratedKey last()
+        {
+            return lastKey;
+        }
+    }
+
+    private SummaryDbUtils()
+    {
+        throw new IllegalStateException(getClass() + " is static utility class and shall not be instantiated");
+    }
+
+    static Summary readSummary(@NotNull TableMetadata metadata, @NotNull SSTable ssTable) throws IOException
+    {
+        try (InputStream in = ssTable.openSummaryStream())
+        {
+            return readSummary(in, metadata.partitioner, metadata.params.minIndexInterval, metadata.params.maxIndexInterval);
+        }
+    }
+
+    /**
+     * Read and deserialize the Summary.db file
+     *
+     * @param summaryStream    input stream for Summary.db file
+     * @param partitioner      token partitioner
+     * @param minIndexInterval min index interval
+     * @param maxIndexInterval max index interval
+     * @return Summary object
+     * @throws IOException io exception
+     */
+    static Summary readSummary(InputStream summaryStream,
+                               IPartitioner partitioner,
+                               int minIndexInterval,
+                               int maxIndexInterval) throws IOException
+    {
+        if (summaryStream == null)
+        {
+            return null;
+        }
+
+        try (DataInputStream is = new DataInputStream(summaryStream))
+        {
+            IndexSummary indexSummary = IndexSummary.serializer.deserialize(is, partitioner, minIndexInterval, maxIndexInterval);
+            DecoratedKey firstKey = partitioner.decorateKey(ByteBufferUtil.readWithLength(is));
+            DecoratedKey lastKey = partitioner.decorateKey(ByteBufferUtil.readWithLength(is));
+            return new Summary(indexSummary, firstKey, lastKey);
+        }
+    }
+
+    public interface TokenList
+    {
+        int size();
+
+        BigInteger tokenAt(int index);
+    }
+
+    /**
+     * Binary search Summary.db to find nearest offset in Index.db that precedes the token we are looking for
+     *
+     * @param summary     IndexSummary from Summary.db file
+     * @param partitioner Cassandra partitioner to hash partition keys to token
+     * @param token       the token we are trying to find
+     * @return offset into the Index.db file for the closest to partition in the Summary.db file that precedes the token we are looking for
+     */
+    public static long findIndexOffsetInSummary(IndexSummary summary, IPartitioner partitioner, BigInteger token)
+    {
+        return summary.getPosition(binarySearchSummary(summary, partitioner, token));
+    }
+
+    public static class IndexSummaryTokenList implements TokenList
+    {
+        final IPartitioner partitioner;
+        final IndexSummary summary;
+
+        IndexSummaryTokenList(IPartitioner partitioner,
+                              IndexSummary summary)
+        {
+            this.partitioner = partitioner;
+            this.summary = summary;
+        }
+
+        public int size()
+        {
+            return summary.size();
+        }
+
+        public BigInteger tokenAt(int index)
+        {
+            return ReaderUtils.tokenToBigInteger(partitioner.decorateKey(ByteBuffer.wrap(summary.getKey(index))).getToken());
+        }
+    }
+
+    public static int binarySearchSummary(IndexSummary summary, IPartitioner partitioner, BigInteger token)
+    {
+        return binarySearchSummary(new IndexSummaryTokenList(partitioner, summary), token);
+    }
+
+    /**
+     * Binary search the Summary.db file to find nearest index offset in Index.db for a given token.
+     * Method lifted from org.apache.cassandra.io.sstable.IndexSummary.binarySearch(PartitionPosition key) and reworked for tokens.
+     *
+     * @param tokenList list of tokens to binary search
+     * @param token     token to find
+     * @return closest offset in Index.db preceding token
+     */
+    public static int binarySearchSummary(TokenList tokenList, BigInteger token)
+    {
+        int low = 0;
+        int mid = tokenList.size();
+        int high = mid - 1;
+        int result = -1;
+        while (low <= high)
+        {
+            mid = low + high >> 1;
+            result = token.compareTo(tokenList.tokenAt(mid));
+            if (result > 0)
+            {
+                low = mid + 1;
+            }
+            else if (result < 0)
+            {
+                high = mid - 1;
+            }
+            else
+            {
+                break;  // Exact match
+            }
+        }
+
+        // If:
+        //  1) result < 0: the token is less than nearest sampled token found at mid, so we need to start from mid - 1.
+        //  2) result == 0: we found an exact match for the token in the sample,
+        //     but there may be token collisions in Data.db so start from mid -1 to be safe.
+        //  3) result > 0: the nearest sample token at mid is less than the token so we can start from that position.
+        return result <= 0 ? Math.max(0, mid - 1) : mid;
+    }
+}
diff --git a/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/sparksql/RangeTombstoneMarkerImplementation.java b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/sparksql/RangeTombstoneMarkerImplementation.java
new file mode 100644
index 0000000..7eed648
--- /dev/null
+++ b/cassandra-four-zero/src/main/java/org/apache/cassandra/spark/sparksql/RangeTombstoneMarkerImplementation.java
@@ -0,0 +1,133 @@
+/*
+ * 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.cassandra.spark.sparksql;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.cassandra.db.ClusteringBound;
+import org.apache.cassandra.db.ClusteringPrefix;
+import org.apache.cassandra.spark.data.CqlField;
+import org.apache.cassandra.spark.data.CqlTable;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+public class RangeTombstoneMarkerImplementation implements RangeTombstoneMarker
+{
+    private final org.apache.cassandra.db.rows.RangeTombstoneMarker marker;
+
+    public RangeTombstoneMarkerImplementation(@NotNull org.apache.cassandra.db.rows.RangeTombstoneMarker marker)
+    {
+        this.marker = marker;
+    }
+
+    @Override
+    public boolean isBoundary()
+    {
+        return marker.isBoundary();
+    }
+
+    @Override
+    public boolean isOpen(boolean value)
+    {
+        return marker.isOpen(value);
+    }
+
+    @Override
+    public boolean isClose(boolean value)
+    {
+        return marker.isClose(value);
+    }
+
+    @Override
+    public long openDeletionTime(boolean value)
+    {
+        return marker.openDeletionTime(value).markedForDeleteAt();
+    }
+
+    @Override
+    public long closeDeletionTime(boolean value)
+    {
+        return marker.closeDeletionTime(value).markedForDeleteAt();
+    }
+
+    @Override
+    @Nullable
+    public Object[] computeRange(@Nullable Object[] range, @NotNull List<InternalRow> list, @NotNull CqlTable table)
+    {
+        if (marker.isBoundary())
+        {
+            Preconditions.checkState(range != null);
+            ClusteringBound<?> close = marker.closeBound(false);
+            range[END_FIELD_POSITION] = buildClusteringKey(table, close.clustering());
+            range[END_INCLUSIVE_FIELD_POSITION] = close.isInclusive();
+            list.add(new GenericInternalRow(range));
+            ClusteringBound<?> open = marker.openBound(false);
+            range = new Object[TOTAL_FIELDS];
+            range[START_FIELD_POSITION] = buildClusteringKey(table, open.clustering());
+            range[START_INCLUSIVE_FIELD_POSITION] = open.isInclusive();
+        }
+        else if (marker.isOpen(false))  // Open bound
+        {
+            Preconditions.checkState(range == null);
+            range = new Object[TOTAL_FIELDS];
+            ClusteringBound<?> open = marker.openBound(false);
+            range[START_FIELD_POSITION] = buildClusteringKey(table, open.clustering());
+            range[START_INCLUSIVE_FIELD_POSITION] = open.isInclusive();
+        }
+        else  // Close bound
+        {
+            Preconditions.checkState(range != null);
+            ClusteringBound<?> close = marker.closeBound(false);
+            range[END_FIELD_POSITION] = buildClusteringKey(table, close.clustering());
+            range[END_INCLUSIVE_FIELD_POSITION] = close.isInclusive();
+            list.add(new GenericInternalRow(range));
+            range = null;
+        }
+        return range;
+    }
+
+    @NotNull
+    private static GenericInternalRow buildClusteringKey(@NotNull CqlTable table,
+                                                         @NotNull ClusteringPrefix<?> clustering)
+    {
+        int index = 0;
+        Object[] ckFields = new Object[table.numClusteringKeys()];
+        for (CqlField field : table.clusteringKeys())
+        {
+            if (index < clustering.size())
+            {
+                ByteBuffer buffer = clustering.bufferAt(index);
+                ckFields[index] = field.deserialize(buffer);
+                index++;
+            }
+            else
+            {
+                // A valid range bound does not non-null values following a null value
+                break;
+            }
+        }
+        return new GenericInternalRow(ckFields);
+    }
+}
diff --git a/cassandra-four-zero/src/test/java/org/apache/cassandra/spark/data/types/DateTypeTests.java b/cassandra-four-zero/src/test/java/org/apache/cassandra/spark/data/types/DateTypeTests.java
new file mode 100644
index 0000000..0acbf2a
--- /dev/null
+++ b/cassandra-four-zero/src/test/java/org/apache/cassandra/spark/data/types/DateTypeTests.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.cassandra.spark.data.types;
+
+import java.time.LocalDate;
+
+import org.junit.Test;
+
+import org.apache.cassandra.bridge.CassandraBridgeImplementation;
+import org.apache.cassandra.serializers.SimpleDateSerializer;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class DateTypeTests
+{
+    private static final CassandraBridgeImplementation BRIDGE = new CassandraBridgeImplementation();
+
+    @Test
+    public void testDateConversion()
+    {
+        int cassandraDate = SimpleDateSerializer.dateStringToDays("2021-07-16");
+        assertTrue(cassandraDate < 0);
+        assertEquals("2021-07-16", SimpleDateSerializer.instance.toString(cassandraDate));
+        Object sparkSqlDate = Date.INSTANCE.toSparkSqlType(cassandraDate, false);
+        assertTrue(sparkSqlDate instanceof Integer);
+        int numDays = (int) sparkSqlDate;
+        assertTrue(numDays > 0);
+        LocalDate end = LocalDate.of(1970, 1, 1)
+                                       .plusDays(numDays);
+        assertEquals(2021, end.getYear());
+        assertEquals(7, end.getMonthValue());
+        assertEquals(16, end.getDayOfMonth());
+        Object cqlWriterObj = Date.INSTANCE.convertForCqlWriter(numDays, BRIDGE.getVersion());
+        org.apache.cassandra.cql3.functions.types.LocalDate cqlWriterDate = (org.apache.cassandra.cql3.functions.types.LocalDate) cqlWriterObj;
+        assertEquals(2021, cqlWriterDate.getYear());
+        assertEquals(7, cqlWriterDate.getMonth());
+        assertEquals(16, cqlWriterDate.getDay());
+    }
+}
diff --git a/cassandra-four-zero/src/test/java/org/apache/cassandra/spark/reader/CdcScannerTests.java b/cassandra-four-zero/src/test/java/org/apache/cassandra/spark/reader/CdcScannerTests.java
new file mode 100644
index 0000000..32fbf17
--- /dev/null
+++ b/cassandra-four-zero/src/test/java/org/apache/cassandra/spark/reader/CdcScannerTests.java
@@ -0,0 +1,152 @@
+/*
+ * 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.cassandra.spark.reader;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+import java.util.concurrent.TimeUnit;
+
+import org.junit.Test;
+
+import org.apache.cassandra.db.commitlog.PartitionUpdateWrapper;
+import org.apache.cassandra.spark.cdc.watermarker.Watermarker;
+import org.apache.cassandra.spark.stats.Stats;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyInt;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+public class CdcScannerTests
+{
+    private Watermarker watermarker(boolean isLate)
+    {
+        Watermarker watermarker = mock(Watermarker.class);
+        when(watermarker.seenBefore(any(PartitionUpdateWrapper.class))).thenReturn(isLate);
+        return watermarker;
+    }
+
+    private static PartitionUpdateWrapper update(long timestamp)
+    {
+        PartitionUpdateWrapper update = mock(PartitionUpdateWrapper.class);
+        when(update.maxTimestampMicros()).thenReturn(timestamp * 1000L);  // In microseconds
+        return update;
+    }
+
+    private void test(List<PartitionUpdateWrapper> updates,
+                      Watermarker watermarker,
+                      int minimumReplicasPerMutation,
+                      boolean shouldPublish)
+    {
+        assertEquals(shouldPublish, CdcScannerBuilder.filter(updates, minimumReplicasPerMutation, watermarker, Stats.DoNothingStats.INSTANCE));
+    }
+
+    @Test
+    public void testPublishedClAll()
+    {
+        Watermarker watermarker = watermarker(false);
+        long now = TimeUnit.MILLISECONDS.toMicros(System.currentTimeMillis());
+        PartitionUpdateWrapper update1 = update(now);
+        PartitionUpdateWrapper update2 = update(now);
+        PartitionUpdateWrapper update3 = update(now);
+        List<PartitionUpdateWrapper> updates = Arrays.asList(update1, update2, update3);
+        test(updates, watermarker, 3, true);
+        for (PartitionUpdateWrapper update : updates)
+        {
+            verify(watermarker, never()).recordReplicaCount(eq(update), anyInt());
+        }
+    }
+
+    @Test
+    public void testPublishedClQuorum()
+    {
+        Watermarker watermarker = watermarker(false);
+        long now = TimeUnit.MILLISECONDS.toMicros(System.currentTimeMillis());
+        PartitionUpdateWrapper update1 = update(now);
+        PartitionUpdateWrapper update2 = update(now);
+        List<PartitionUpdateWrapper> updates = Arrays.asList(update1, update2);
+        test(updates, watermarker, 2, true);
+        for (PartitionUpdateWrapper update : updates)
+        {
+            verify(watermarker, never()).recordReplicaCount(eq(update), anyInt());
+        }
+    }
+
+    @Test
+    public void testInsufficientReplicas()
+    {
+        Watermarker watermarker = watermarker(false);
+        long now = TimeUnit.MILLISECONDS.toMicros(System.currentTimeMillis());
+        PartitionUpdateWrapper update1 = update(now);
+        List<PartitionUpdateWrapper> updates = Collections.singletonList(update1);
+        test(updates, watermarker, 2, false);
+        for (PartitionUpdateWrapper update : updates)
+        {
+            verify(watermarker).recordReplicaCount(eq(update), eq(1));
+        }
+    }
+
+    @Test
+    public void testInsufficientReplicasLate()
+    {
+        Watermarker watermarker = watermarker(false);
+        long now = TimeUnit.MILLISECONDS.toMicros(System.currentTimeMillis());
+        PartitionUpdateWrapper update1 = update(now);
+        List<PartitionUpdateWrapper> updates = Collections.singletonList(update1);
+        test(updates, watermarker, 2, false);
+        for (PartitionUpdateWrapper update : updates)
+        {
+            verify(watermarker).recordReplicaCount(eq(update), eq(1));
+        }
+    }
+
+    @Test
+    public void testLateMutation()
+    {
+        Watermarker watermarker = watermarker(true);
+        long now = TimeUnit.MILLISECONDS.toMicros(System.currentTimeMillis());
+        PartitionUpdateWrapper update1 = update(now);
+        PartitionUpdateWrapper update2 = update(now);
+        List<PartitionUpdateWrapper> updates = Arrays.asList(update1, update2);
+        test(updates, watermarker, 2, true);
+        verify(watermarker).untrackReplicaCount(eq(update1));
+    }
+
+    @Test
+    public void testCommitLogFilename()
+    {
+        assertEquals(12345L, Objects.requireNonNull(CdcScannerBuilder.extractSegmentId("CommitLog-4-12345.log")).longValue());
+        assertEquals(12345L, Objects.requireNonNull(CdcScannerBuilder.extractSegmentId("CommitLog-12345.log")).longValue());
+        assertEquals(1646094405659L, Objects.requireNonNull(CdcScannerBuilder.extractSegmentId("CommitLog-7-1646094405659.log")).longValue());
+        assertEquals(1646094405659L, Objects.requireNonNull(CdcScannerBuilder.extractSegmentId("CommitLog-1646094405659.log")).longValue());
+        assertEquals(1646094405659L, Objects.requireNonNull(CdcScannerBuilder.extractSegmentId("CommitLog-242-1646094405659.log")).longValue());
+        assertNull(CdcScannerBuilder.extractSegmentId("CommitLog-123-abcd.log"));
+        assertNull(CdcScannerBuilder.extractSegmentId("CommitLog-abcd.log"));
+        assertNull(CdcScannerBuilder.extractSegmentId("CommitLog.log"));
+        assertNull(CdcScannerBuilder.extractSegmentId("abcd"));
+    }
+}
diff --git a/cassandra-four-zero/src/test/java/org/apache/cassandra/spark/reader/IndexDbTests.java b/cassandra-four-zero/src/test/java/org/apache/cassandra/spark/reader/IndexDbTests.java
new file mode 100644
index 0000000..bc75155
--- /dev/null
+++ b/cassandra-four-zero/src/test/java/org/apache/cassandra/spark/reader/IndexDbTests.java
@@ -0,0 +1,281 @@
+/*
+ * 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.cassandra.spark.reader;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.EOFException;
+import java.io.IOException;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import com.google.common.base.Preconditions;
+import org.junit.Test;
+
+import org.apache.cassandra.bridge.CassandraBridgeImplementation;
+import org.apache.cassandra.bridge.TokenRange;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.spark.data.FileType;
+import org.apache.cassandra.spark.data.SSTable;
+import org.apache.cassandra.spark.data.partitioner.Partitioner;
+import org.apache.cassandra.spark.stats.Stats;
+import org.apache.cassandra.spark.utils.TemporaryDirectory;
+import org.apache.cassandra.spark.utils.test.TestSSTable;
+import org.apache.cassandra.spark.utils.test.TestSchema;
+import org.jetbrains.annotations.NotNull;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.quicktheories.QuickTheory.qt;
+import static org.quicktheories.generators.Generate.constant;
+import static org.quicktheories.generators.SourceDSL.arbitrary;
+import static org.quicktheories.generators.SourceDSL.integers;
+import static org.quicktheories.generators.SourceDSL.maps;
+
+public class IndexDbTests
+{
+    private static final CassandraBridgeImplementation BRIDGE = new CassandraBridgeImplementation();
+
+    private static final class IndexRow implements Comparable<IndexRow>
+    {
+        private final BigInteger token;
+        private final int value;
+        private int position = 0;
+
+        IndexRow(IPartitioner partitioner, int value)
+        {
+            this.token = token(partitioner, value);
+            this.value = value;
+        }
+
+        public int compareTo(@NotNull IndexRow that)
+        {
+            return this.token.compareTo(that.token);
+        }
+    }
+
+    @Test
+    @SuppressWarnings("static-access")
+    public void testSearchIndex()
+    {
+        qt().forAll(arbitrary().enumValues(Partitioner.class))
+            .checkAssert(partitioner -> {
+                try (TemporaryDirectory directory = new TemporaryDirectory())
+                {
+                    TestSchema schema = TestSchema.basicBuilder(BRIDGE).withCompression(false).build();
+                    IPartitioner iPartitioner = BRIDGE.getPartitioner(partitioner);
+                    int numRows = 5000;
+
+                    // Write an SSTable and record token
+                    List<BigInteger> tokens = new ArrayList<>(numRows);
+                    schema.writeSSTable(directory, BRIDGE, partitioner, writer -> {
+                        for (int row = 0; row < numRows; row++)
+                        {
+                            ByteBuffer key = (ByteBuffer) ByteBuffer.allocate(4).putInt(row).flip();
+                            BigInteger token = ReaderUtils.tokenToBigInteger(iPartitioner.decorateKey(key).getToken());
+                            tokens.add(token);
+                            writer.write(row, 0, row);
+                        }
+                    });
+                    assertEquals(1, TestSSTable.countIn(directory.path()));
+                    Collections.sort(tokens);
+
+                    TableMetadata metadata = Schema.instance.getTableMetadata(schema.keyspace, schema.table);
+                    assertNotNull("Could not find table metadata", metadata);
+
+                    Path summaryDb = TestSSTable.firstIn(directory.path(), FileType.SUMMARY);
+                    assertNotNull("Could not find summary", summaryDb);
+
+                    SSTable ssTable = TestSSTable.firstIn(directory.path());
+                    assertNotNull("Could not find SSTable", ssTable);
+
+                    int rowSize = 39;
+                    int sample = 4;
+                    // Sample the token list and read offset in Index.db for sampled list and verify the
+                    // offset matches the expected we sample the list as IndexDbUtils.findStartOffset(...)
+                    // returns the previous offset, so we want to test it correctly skips tokens less than
+                    // the token we are looking for before returning
+                    List<BigInteger> sparseList = IntStream.range(0, tokens.size())
+                                                           .filter(index -> index > 0 && index % sample == 0)
+                                                           .mapToObj(tokens::get)
+                                                           .collect(Collectors.toList());
+                    assertEquals((numRows / 4) - 1, sparseList.size());
+                    try (DataInputStream in = new DataInputStream(Objects.requireNonNull(ssTable.openPrimaryIndexStream())))
+                    {
+                        try
+                        {
+                            for (int index = 0; index < sparseList.size(); index++)
+                            {
+                                BigInteger token = sparseList.get(index);
+                                long expectedOffset = (((index + 1L) * sample) - 1) * rowSize;
+                                long offset = IndexDbUtils.findStartOffset(in,
+                                                                           iPartitioner,
+                                                                           TokenRange.singleton(token),
+                                                                           Stats.DoNothingStats.INSTANCE);
+                                assertEquals(expectedOffset, offset);
+                                ReaderUtils.skipRowIndexEntry(in);
+                            }
+                        }
+                        catch (EOFException ignore)
+                        {
+                        }
+                    }
+                }
+                catch (IOException exception)
+                {
+                    throw new RuntimeException(exception);
+                }
+            });
+    }
+
+    @Test
+    @SuppressWarnings("static-access")
+    public void testFindStartEndOffset()
+    {
+        int numValues = 5000;
+        qt().forAll(arbitrary().enumValues(Partitioner.class),                           // Partitioner
+                    maps().of(integers().allPositive(), constant(1)).ofSize(numValues),  // Unique keys (ignore values)
+                    integers().between(1, numValues - 1))                                // Start position
+            .checkAssert((partitioner, rawValues, startPos) -> {
+                    IPartitioner iPartitioner = BRIDGE.getPartitioner(partitioner);
+                    int rowSize = 256;
+
+                    // Generate random index row values and sort by token
+                    IndexRow[] rows = rawValues.keySet().stream()
+                                                        .map(value -> new IndexRow(iPartitioner, value))
+                                                        .sorted()
+                                                        .toArray(IndexRow[]::new);
+                    IntStream.range(0, rows.length).forEach(index -> rows[index].position = index * rowSize);  // Update position offset
+                    IndexRow startRow = rows[startPos];
+                    int[] valuesAndOffsets = Arrays.stream(rows)
+                                                   .map(row -> new int[]{row.value, row.position})
+                                                   .flatMapToInt(Arrays::stream)
+                                                   .toArray();
+
+                    try (DataInputStream in = mockDataInputStream(valuesAndOffsets))
+                    {
+                        long startOffset = IndexDbUtils.findStartOffset(in,
+                                                                        iPartitioner,
+                                                                        TokenRange.singleton(startRow.token),
+                                                                        Stats.DoNothingStats.INSTANCE);
+                        assertEquals(rows[startPos - 1].position, startOffset);
+                        ReaderUtils.skipRowIndexEntry(in);
+                    }
+                    catch (IOException exception)
+                    {
+                        throw new RuntimeException(exception);
+                    }
+                });
+    }
+
+    @Test
+    @SuppressWarnings("static-access")
+    public void testReadToken()
+    {
+        qt().withExamples(500)
+            .forAll(arbitrary().enumValues(Partitioner.class), integers().all())
+            .checkAssert((partitioner, value) -> {
+                IPartitioner iPartitioner = BRIDGE.getPartitioner(partitioner);
+                BigInteger expectedToken = token(iPartitioner, value);
+                try (DataInputStream in = mockDataInputStream(value, 0))
+                {
+                    IndexDbUtils.readNextToken(iPartitioner, in, new Stats()
+                    {
+                        public void readPartitionIndexDb(ByteBuffer key, BigInteger token)
+                        {
+                            assertEquals(value.intValue(), key.getInt());
+                            assertEquals(expectedToken, token);
+                        }
+                    });
+                }
+                catch (IOException exception)
+                {
+                    throw new RuntimeException(exception);
+                }
+            }
+        );
+    }
+
+    @Test
+    public void testLessThan()
+    {
+        assertTrue(IndexDbUtils.isLessThan(BigInteger.valueOf(4L), TokenRange.open(BigInteger.valueOf(5L), BigInteger.valueOf(10L))));
+        assertTrue(IndexDbUtils.isLessThan(BigInteger.valueOf(4L), TokenRange.openClosed(BigInteger.valueOf(5L), BigInteger.valueOf(10L))));
+        assertTrue(IndexDbUtils.isLessThan(BigInteger.valueOf(4L), TokenRange.closed(BigInteger.valueOf(5L), BigInteger.valueOf(10L))));
+        assertTrue(IndexDbUtils.isLessThan(BigInteger.valueOf(4L), TokenRange.closedOpen(BigInteger.valueOf(5L), BigInteger.valueOf(10L))));
+
+        assertTrue(IndexDbUtils.isLessThan(BigInteger.valueOf(5L), TokenRange.open(BigInteger.valueOf(5L), BigInteger.valueOf(10L))));
+        assertTrue(IndexDbUtils.isLessThan(BigInteger.valueOf(5L), TokenRange.openClosed(BigInteger.valueOf(5L), BigInteger.valueOf(10L))));
+        assertFalse(IndexDbUtils.isLessThan(BigInteger.valueOf(5L), TokenRange.closed(BigInteger.valueOf(5L), BigInteger.valueOf(10L))));
+        assertFalse(IndexDbUtils.isLessThan(BigInteger.valueOf(5L), TokenRange.closedOpen(BigInteger.valueOf(5L), BigInteger.valueOf(10L))));
+
+        assertFalse(IndexDbUtils.isLessThan(BigInteger.valueOf(6L), TokenRange.open(BigInteger.valueOf(5L), BigInteger.valueOf(10L))));
+        assertFalse(IndexDbUtils.isLessThan(BigInteger.valueOf(6L), TokenRange.openClosed(BigInteger.valueOf(5L), BigInteger.valueOf(10L))));
+        assertFalse(IndexDbUtils.isLessThan(BigInteger.valueOf(6L), TokenRange.closed(BigInteger.valueOf(5L), BigInteger.valueOf(10L))));
+        assertFalse(IndexDbUtils.isLessThan(BigInteger.valueOf(6L), TokenRange.closedOpen(BigInteger.valueOf(5L), BigInteger.valueOf(10L))));
+    }
+
+    private static BigInteger token(IPartitioner iPartitioner, int value)
+    {
+        // Cast to ByteBuffer required when compiling with Java 8
+        return ReaderUtils.tokenToBigInteger(iPartitioner.decorateKey((ByteBuffer) ByteBuffer.allocate(4).putInt(value).flip()).getToken());
+    }
+
+    // Creates an in-memory DataInputStream mocking Index.db bytes, with length (short), key (int), position (vint)
+    private static DataInputStream mockDataInputStream(int... valuesAndOffsets) throws IOException
+    {
+        Preconditions.checkArgument(valuesAndOffsets.length % 2 == 0);
+        int numValues = valuesAndOffsets.length / 2;
+
+        int size = (numValues * 7);  // 2 bytes short length, 4 bytes partition key value, 1 byte promoted index
+        size += IntStream.range(0, valuesAndOffsets.length)  // Variable int for position offset
+                         .filter(index -> (index + 1) % 2 == 0)
+                         .map(index -> valuesAndOffsets[index])
+                         .map(ReaderUtils::vIntSize)
+                         .sum();
+
+        ByteBuffer buffer = ByteBuffer.allocate(size);
+        for (int index = 0; index < valuesAndOffsets.length; index += 2)
+        {
+            buffer.putShort((short) 4)
+               .putInt(valuesAndOffsets[index]);  // Value
+            ReaderUtils.writePosition(valuesAndOffsets[index + 1], buffer);  // Write variable int position offset
+            ReaderUtils.writePosition(0L, buffer);  // Promoted index
+        }
+
+        buffer.flip();
+        byte[] bytes = new byte[buffer.remaining()];
+        buffer.get(bytes);
+
+        return new DataInputStream(new ByteArrayInputStream(bytes));
+    }
+}
diff --git a/cassandra-four-zero/src/test/java/org/apache/cassandra/spark/reader/IndexOffsetTests.java b/cassandra-four-zero/src/test/java/org/apache/cassandra/spark/reader/IndexOffsetTests.java
new file mode 100644
index 0000000..f56164b
--- /dev/null
+++ b/cassandra-four-zero/src/test/java/org/apache/cassandra/spark/reader/IndexOffsetTests.java
@@ -0,0 +1,190 @@
+/*
+ * 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.cassandra.spark.reader;
+
+import java.io.IOException;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+
+import com.google.common.collect.ImmutableMultimap;
+import com.google.common.collect.Multimap;
+import org.apache.commons.lang.mutable.MutableInt;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.bridge.CassandraBridgeImplementation;
+import org.apache.cassandra.bridge.TokenRange;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.spark.data.SSTable;
+import org.apache.cassandra.spark.data.partitioner.Partitioner;
+import org.apache.cassandra.spark.sparksql.filters.SparkRangeFilter;
+import org.apache.cassandra.spark.stats.Stats;
+import org.apache.cassandra.spark.utils.TemporaryDirectory;
+import org.apache.cassandra.spark.utils.test.TestSSTable;
+import org.apache.cassandra.spark.utils.test.TestSchema;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.quicktheories.QuickTheory.qt;
+import static org.quicktheories.generators.SourceDSL.arbitrary;
+import static org.quicktheories.generators.SourceDSL.booleans;
+
+public class IndexOffsetTests
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(IndexOffsetTests.class);
+    private static final CassandraBridgeImplementation BRIDGE = new CassandraBridgeImplementation();
+    @SuppressWarnings("unchecked")
+    private static final Multimap<Partitioner, TokenRange> RANGES = new ImmutableMultimap.Builder<Partitioner, TokenRange>()
+        .putAll(Partitioner.RandomPartitioner,  TokenRange.closedOpen(BigInteger.ZERO,
+                                                                      BigInteger.ZERO),
+                                                TokenRange.closedOpen(BigInteger.ONE,
+                                                                      new BigInteger("56713727820156410577229101238628035242")),
+                                                TokenRange.closedOpen(new BigInteger("56713727820156410577229101238628035243"),
+                                                                      new BigInteger("113427455640312821154458202477256070484")),
+                                                TokenRange.closedOpen(new BigInteger("113427455640312821154458202477256070485"),
+                                                                      new BigInteger("170141183460469231731687303715884105727")))
+        .putAll(Partitioner.Murmur3Partitioner, TokenRange.closed(new BigInteger("-9223372036854775808"),
+                                                                  new BigInteger("-9223372036854775808")),
+                                                TokenRange.closed(new BigInteger("-9223372036854775807"),
+                                                                  new BigInteger("-3074457345618258603")),
+                                                TokenRange.closed(new BigInteger("-3074457345618258602"),
+                                                                  new BigInteger("3074457345618258602")),
+                                                TokenRange.closed(new BigInteger("3074457345618258603"),
+                                                                  new BigInteger("9223372036854775807")))
+        .build();
+
+    @Test
+    @SuppressWarnings("static-access")
+    public void testReadIndexOffsets()
+    {
+        qt().forAll(arbitrary().enumValues(Partitioner.class), booleans().all())
+            .checkAssert((partitioner, enableCompression) -> {
+                try (TemporaryDirectory directory = new TemporaryDirectory())
+                {
+                    int numKeys = 100000;
+                    TestSchema schema = TestSchema.basicBuilder(BRIDGE)
+                                                  .withCompression(enableCompression)
+                                                  .build();
+
+                    schema.writeSSTable(directory, BRIDGE, partitioner, writer -> {
+                        for (int index = 0; index < numKeys; index++)
+                        {
+                            writer.write(index, 0, index);
+                        }
+                    });
+                    assertEquals(1, TestSSTable.countIn(directory.path()));
+
+                    TableMetadata metadata = Schema.instance.getTableMetadata(schema.keyspace, schema.table);
+                    assertNotNull("Could not find table metadata", metadata);
+
+                    SSTable ssTable = TestSSTable.firstIn(directory.path());
+                    assertNotNull("Could not find SSTable", ssTable);
+
+                    Collection<TokenRange> ranges = RANGES.get(partitioner);
+                    assertNotNull("Unknown paritioner", ranges);
+
+                    LOGGER.info("Testing index offsets numKeys={} sparkPartitions={} partitioner={} enableCompression={}",
+                                numKeys, ranges.size(), partitioner.name(), enableCompression);
+
+                    MutableInt skipped = new MutableInt(0);
+                    int[] counts = new int[numKeys];
+                    for (TokenRange range : ranges)
+                    {
+                        SSTableReader reader = SSTableReader.builder(metadata, ssTable)
+                                                            .withSparkRangeFilter(SparkRangeFilter.create(range))
+                                                            .withStats(new Stats()
+                                                            {
+                                                                public void skippedPartition(ByteBuffer key, BigInteger token)
+                                                                {
+                                                                    skipped.add(1);
+                                                                }
+                                                            })
+                                                            .build();
+                        if (reader.ignore())
+                        {
+                            // We can skip this range entirely, it doesn't overlap with SSTable
+                            continue;
+                        }
+
+                        // Iterate through SSTable partitions,
+                        // each scanner should only read tokens within its own token range
+                        try (ISSTableScanner scanner = reader.scanner())
+                        {
+                            while (scanner.hasNext())
+                            {
+                                UnfilteredRowIterator rowIterator = scanner.next();
+                                int key = rowIterator.partitionKey().getKey().getInt();
+                                // Count how many times we read a key across all 'spark' token partitions
+                                counts[key]++;
+                                while (rowIterator.hasNext())
+                                {
+                                    rowIterator.next();
+                                }
+                            }
+                        }
+                    }
+
+                    // Verify we read each key exactly once across all Spark partitions
+                    assertEquals(counts.length, numKeys);
+                    int index = 0;
+                    for (int count : counts)
+                    {
+                        if (count == 0)
+                        {
+                            LOGGER.error("Missing key key={} token={} partitioner={}",
+                                         index,
+                                         // Cast to ByteBuffer required when compiling with Java 8
+                                         ReaderUtils.tokenToBigInteger(BRIDGE
+                                                .getPartitioner(partitioner)
+                                                .decorateKey((ByteBuffer) ByteBuffer.allocate(4).putInt(index).flip())
+                                                .getToken()),
+                                         partitioner.name());
+                        }
+                        else if (count > 1)
+                        {
+                            LOGGER.error("Key read by more than 1 Spark partition key={} token={} partitioner={}",
+                                         index,
+                                         // Cast to ByteBuffer required when compiling with Java 8
+                                         ReaderUtils.tokenToBigInteger(BRIDGE
+                                                .getPartitioner(partitioner)
+                                                .decorateKey((ByteBuffer) ByteBuffer.allocate(4).putInt(index).flip())
+                                                .getToken()),
+                                         partitioner.name());
+                        }
+                        assertEquals(count > 0 ? "Key " + index + " read " + count + " times"
+                                               : "Key not found: " + index, 1, count);
+                        index++;
+                    }
+
+                    LOGGER.info("Success skippedKeys={} partitioner={}",
+                                skipped.intValue(), partitioner.name());
+                }
+                catch (IOException exception)
+                {
+                    throw new RuntimeException(exception);
+                }
+            });
+    }
+}
diff --git a/cassandra-four-zero/src/test/java/org/apache/cassandra/spark/reader/KryoSerializationTests.java b/cassandra-four-zero/src/test/java/org/apache/cassandra/spark/reader/KryoSerializationTests.java
new file mode 100644
index 0000000..cb22c9f
--- /dev/null
+++ b/cassandra-four-zero/src/test/java/org/apache/cassandra/spark/reader/KryoSerializationTests.java
@@ -0,0 +1,102 @@
+/*
+ * 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.cassandra.spark.reader;
+
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.collect.ImmutableMap;
+import org.junit.Test;
+
+import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+import org.apache.cassandra.bridge.CassandraBridgeImplementation;
+import org.apache.cassandra.db.Clustering;
+import org.apache.cassandra.db.commitlog.PartitionUpdateWrapper;
+import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.db.rows.BTreeRow;
+import org.apache.cassandra.db.rows.BufferCell;
+import org.apache.cassandra.db.rows.Row;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.serializers.LongSerializer;
+import org.apache.cassandra.serializers.UTF8Serializer;
+import org.apache.cassandra.serializers.UUIDSerializer;
+import org.apache.cassandra.spark.data.ReplicationFactor;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+public class KryoSerializationTests
+{
+    private static final CassandraBridgeImplementation BRIDGE = new CassandraBridgeImplementation();
+    private static final Kryo KRYO = new Kryo();
+
+    static
+    {
+        BRIDGE.kryoRegister(KRYO);
+    }
+
+    @Test
+    public void testCdcUpdate()
+    {
+        ReplicationFactor replicationFactor = new ReplicationFactor(ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy,
+                                                                    ImmutableMap.of("DC1", 3, "DC2", 3));
+        BRIDGE.buildSchema("CREATE TABLE cdc.cdc_serialize_test (\n"
+                         + "    a uuid PRIMARY KEY,\n"
+                         + "    b bigint,\n"
+                         + "    c text\n"
+                         + ");", "cdc", replicationFactor);
+        TableMetadata table = Schema.instance.getTableMetadata("cdc", "cdc_serialize_test");
+        long now = System.currentTimeMillis();
+        Row.Builder row = BTreeRow.unsortedBuilder();
+        row.newRow(Clustering.EMPTY);
+        row.addCell(BufferCell.live(table.getColumn(ByteBuffer.wrap("b".getBytes(StandardCharsets.UTF_8))),
+                                    TimeUnit.MILLISECONDS.toMicros(now),
+                                    LongSerializer.instance.serialize(1010101L)));
+        row.addCell(BufferCell.live(table.getColumn(ByteBuffer.wrap("c".getBytes(StandardCharsets.UTF_8))),
+                                    TimeUnit.MILLISECONDS.toMicros(now),
+                                    UTF8Serializer.instance.serialize("some message")));
+        PartitionUpdate partitionUpdate = PartitionUpdate.singleRowUpdate(table,
+                                                                          UUIDSerializer.instance.serialize(UUID.randomUUID()),
+                                                                          row.build());
+        PartitionUpdateWrapper update = new PartitionUpdateWrapper(table,
+                                                                   partitionUpdate,
+                                                                   TimeUnit.MILLISECONDS.toMicros(now));
+        PartitionUpdateWrapper.Serializer serializer = new PartitionUpdateWrapper.Serializer(table);
+        KRYO.register(PartitionUpdateWrapper.class, serializer);
+
+        try (Output out = new Output(1024, -1))
+        {
+            // Serialize and deserialize the update and verify it matches
+            KRYO.writeObject(out, update, serializer);
+            PartitionUpdateWrapper deserialized = KRYO.readObject(new Input(out.getBuffer(), 0, out.position()),
+                                                                  PartitionUpdateWrapper.class,
+                                                                  serializer);
+            assertNotNull(deserialized);
+            assertEquals(update, deserialized);
+            assertEquals(update.digest(), deserialized.digest());
+            assertEquals(update.maxTimestampMicros(), deserialized.maxTimestampMicros());
+        }
+    }
+}
diff --git a/cassandra-four-zero/src/test/java/org/apache/cassandra/spark/reader/PartitionKeyTests.java b/cassandra-four-zero/src/test/java/org/apache/cassandra/spark/reader/PartitionKeyTests.java
new file mode 100644
index 0000000..f31cedd
--- /dev/null
+++ b/cassandra-four-zero/src/test/java/org/apache/cassandra/spark/reader/PartitionKeyTests.java
@@ -0,0 +1,94 @@
+/*
+ * 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.cassandra.spark.reader;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import org.junit.Test;
+
+import org.apache.cassandra.bridge.CassandraBridgeImplementation;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.CompositeType;
+import org.apache.cassandra.spark.data.CqlTable;
+import org.apache.cassandra.spark.data.CqlType;
+import org.apache.cassandra.spark.utils.ComparisonUtils;
+import org.apache.cassandra.spark.utils.test.TestSchema;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.quicktheories.QuickTheory.qt;
+import static org.quicktheories.generators.SourceDSL.arbitrary;
+
+public class PartitionKeyTests
+{
+    private static final CassandraBridgeImplementation BRIDGE = new CassandraBridgeImplementation();
+
+    @Test
+    @SuppressWarnings("static-access")
+    public void testBuildPartitionKey()
+    {
+        qt().forAll(arbitrary().pick(BRIDGE.supportedTypes())).checkAssert(partitionKeyType -> {
+            CqlTable table = TestSchema.builder()
+                                       .withPartitionKey("a", partitionKeyType)
+                                       .withClusteringKey("b", BRIDGE.aInt())
+                                       .withColumn("c", BRIDGE.aInt())
+                                       .build()
+                                       .buildTable();
+            Object value = partitionKeyType.randomValue(100);
+            String string = ((CqlType) partitionKeyType).serializer().toString(value);
+            ByteBuffer buffer = BRIDGE.buildPartitionKey(table, Collections.singletonList(string));
+            assertTrue(ComparisonUtils.equals(value, partitionKeyType.toTestRowType(partitionKeyType.deserialize(buffer))));
+        });
+    }
+
+    @Test
+    @SuppressWarnings("static-access")
+    public void testBuildCompositePartitionKey()
+    {
+        qt().forAll(arbitrary().pick(BRIDGE.supportedTypes())).checkAssert(partitionKeyType -> {
+            CqlTable table = TestSchema.builder()
+                                       .withPartitionKey("a", BRIDGE.aInt())
+                                       .withPartitionKey("b", partitionKeyType)
+                                       .withPartitionKey("c", BRIDGE.text())
+                                       .withClusteringKey("d", BRIDGE.aInt())
+                                       .withColumn("e", BRIDGE.aInt())
+                                       .build()
+                                       .buildTable();
+            List<AbstractType<?>> partitionKeyColumnTypes = BRIDGE.partitionKeyColumnTypes(table);
+            CompositeType compositeType = CompositeType.getInstance(partitionKeyColumnTypes);
+
+            int columnA = (int) BRIDGE.aInt().randomValue(1024);
+            Object columnB = partitionKeyType.randomValue(1024);
+            String columnBString = ((CqlType) partitionKeyType).serializer().toString(columnB);
+            String columnC = (String) BRIDGE.text().randomValue(1024);
+
+            ByteBuffer buffer = BRIDGE.buildPartitionKey(table, Arrays.asList(Integer.toString(columnA), columnBString, columnC));
+            ByteBuffer[] buffers = compositeType.split(buffer);
+            assertEquals(3, buffers.length);
+
+            assertEquals(columnA, buffers[0].getInt());
+            assertEquals(columnB, partitionKeyType.toTestRowType(partitionKeyType.deserialize(buffers[1])));
+            assertEquals(columnC, BRIDGE.text().toTestRowType(BRIDGE.text().deserialize(buffers[2])));
+        });
+    }
+}
diff --git a/cassandra-four-zero/src/test/java/org/apache/cassandra/spark/reader/ReaderUtilsTests.java b/cassandra-four-zero/src/test/java/org/apache/cassandra/spark/reader/ReaderUtilsTests.java
new file mode 100644
index 0000000..c81e408
--- /dev/null
+++ b/cassandra-four-zero/src/test/java/org/apache/cassandra/spark/reader/ReaderUtilsTests.java
@@ -0,0 +1,367 @@
+/*
+ * 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.cassandra.spark.reader;
+
+import java.io.BufferedInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.junit.Test;
+
+import org.apache.cassandra.bridge.CassandraBridgeImplementation;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.SerializationHeader;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.dht.RandomPartitioner;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.metadata.CompactionMetadata;
+import org.apache.cassandra.io.sstable.metadata.MetadataComponent;
+import org.apache.cassandra.io.sstable.metadata.MetadataType;
+import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
+import org.apache.cassandra.io.sstable.metadata.ValidationMetadata;
+import org.apache.cassandra.spark.data.FileType;
+import org.apache.cassandra.spark.data.SSTable;
+import org.apache.cassandra.spark.data.partitioner.Partitioner;
+import org.apache.cassandra.spark.sparksql.filters.PartitionKeyFilter;
+import org.apache.cassandra.spark.utils.TemporaryDirectory;
+import org.apache.cassandra.spark.utils.test.TestSSTable;
+import org.apache.cassandra.spark.utils.test.TestSchema;
+import org.apache.cassandra.utils.Pair;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+import static org.quicktheories.QuickTheory.qt;
+import static org.quicktheories.generators.SourceDSL.arbitrary;
+
+public class ReaderUtilsTests
+{
+    private static final CassandraBridgeImplementation BRIDGE = new CassandraBridgeImplementation();
+    private static final int ROWS = 50;
+    private static final int COLUMNS = 25;
+
+    @Test
+    public void testReadStatsMetaData()
+    {
+        qt().forAll(arbitrary().enumValues(Partitioner.class))
+            .checkAssert(partitioner -> {
+                try (TemporaryDirectory directory = new TemporaryDirectory())
+                {
+                    // Write an SSTable
+                    TestSchema schema = TestSchema.basic(BRIDGE);
+                    long nowMicros = System.currentTimeMillis() * 1000;
+                    schema.writeSSTable(directory, BRIDGE, partitioner, writer -> {
+                        for (int row = 0; row < ROWS; row++)
+                        {
+                            for (int column = 0; column < COLUMNS; column++)
+                            {
+                                writer.write(row, column, row + column);
+                            }
+                        }
+                    });
+                    assertEquals(1, TestSSTable.countIn(directory.path()));
+
+                    String dataFile = TestSSTable.firstIn(directory.path()).getDataFileName();
+                    Descriptor descriptor = Descriptor.fromFilename(
+                            new File(String.format("./%s/%s", schema.keyspace, schema.table), dataFile));
+                    Path statsFile = TestSSTable.firstIn(directory.path(), FileType.STATISTICS);
+
+                    // Deserialize stats meta data and verify components match expected values
+                    Map<MetadataType, MetadataComponent> componentMap;
+                    try (InputStream in = new BufferedInputStream(Files.newInputStream(statsFile)))
+                    {
+                        componentMap = ReaderUtils.deserializeStatsMetadata(in, EnumSet.allOf(MetadataType.class), descriptor);
+                    }
+                    assertNotNull(componentMap);
+                    assertFalse(componentMap.isEmpty());
+
+                    ValidationMetadata validationMetadata = (ValidationMetadata) componentMap.get(MetadataType.VALIDATION);
+                    assertEquals("org.apache.cassandra.dht." + partitioner.name(), validationMetadata.partitioner);
+
+                    CompactionMetadata compactionMetadata = (CompactionMetadata) componentMap.get(MetadataType.COMPACTION);
+                    assertNotNull(compactionMetadata);
+
+                    StatsMetadata statsMetadata = (StatsMetadata) componentMap.get(MetadataType.STATS);
+                    assertEquals(ROWS * COLUMNS, statsMetadata.totalRows);
+                    assertEquals(0L, statsMetadata.repairedAt);
+                    // Want to avoid test flakiness but timestamps should be in same ballpark
+                    long tolerance = TimeUnit.MICROSECONDS.convert(10, TimeUnit.SECONDS);
+                    assertTrue(Math.abs(statsMetadata.maxTimestamp - nowMicros) < tolerance);
+                    assertTrue(Math.abs(statsMetadata.minTimestamp - nowMicros) < tolerance);
+
+                    SerializationHeader.Component header = (SerializationHeader.Component) componentMap.get(MetadataType.HEADER);
+                    assertNotNull(header);
+                    assertEquals("org.apache.cassandra.db.marshal.Int32Type", header.getKeyType().toString());
+                    List<AbstractType<?>> clusteringTypes = header.getClusteringTypes();
+                    assertEquals(1, clusteringTypes.size());
+                    assertEquals("org.apache.cassandra.db.marshal.Int32Type", clusteringTypes.get(0).toString());
+                    assertTrue(header.getStaticColumns().isEmpty());
+                    List<AbstractType<?>> regulars = new ArrayList<>(header.getRegularColumns().values());
+                    assertEquals(1, regulars.size());
+                    assertEquals("org.apache.cassandra.db.marshal.Int32Type", regulars.get(0).toString());
+                }
+                catch (IOException exception)
+                {
+                    throw new RuntimeException(exception);
+                }
+            });
+    }
+
+    @Test
+    public void testReadFirstLastPartitionKey()
+    {
+        qt().forAll(arbitrary().enumValues(Partitioner.class))
+            .checkAssert(partitioner -> {
+                try (TemporaryDirectory directory = new TemporaryDirectory())
+                {
+                    // Write an SSTable
+                    TestSchema schema = TestSchema.basic(BRIDGE);
+                    schema.writeSSTable(directory, BRIDGE, partitioner, writer -> {
+                        for (int row = 0; row < ROWS; row++)
+                        {
+                            for (int column = 0; column < COLUMNS; column++)
+                            {
+                                writer.write(row, column, row + column);
+                            }
+                        }
+                    });
+                    assertEquals(1, TestSSTable.countIn(directory.path()));
+
+                    // Read Summary.db file for first and last partition keys from Summary.db
+                    Path summaryFile = TestSSTable.firstIn(directory.path(), FileType.SUMMARY);
+                    SummaryDbUtils.Summary summaryKeys;
+                    try (InputStream in = new BufferedInputStream(Files.newInputStream(summaryFile)))
+                    {
+                        summaryKeys = SummaryDbUtils.readSummary(in, Murmur3Partitioner.instance, 128, 2048);
+                    }
+                    assertNotNull(summaryKeys);
+                    assertNotNull(summaryKeys.first());
+                    assertNotNull(summaryKeys.last());
+
+                    // Read Primary Index.db file for first and last partition keys from Summary.db
+                    Path indexFile = TestSSTable.firstIn(directory.path(), FileType.INDEX);
+                    Pair<DecoratedKey, DecoratedKey> indexKeys;
+                    try (InputStream in = new BufferedInputStream(Files.newInputStream(indexFile)))
+                    {
+                        Pair<ByteBuffer, ByteBuffer> keys = ReaderUtils.readPrimaryIndex(in, true, Collections.emptyList());
+                        indexKeys = Pair.create(Murmur3Partitioner.instance.decorateKey(keys.left),
+                                                Murmur3Partitioner.instance.decorateKey(keys.right));
+                    }
+                    assertNotNull(indexKeys);
+                    assertEquals(indexKeys.left, summaryKeys.first());
+                    assertEquals(indexKeys.right, summaryKeys.last());
+                }
+                catch (IOException exception)
+                {
+                    throw new RuntimeException(exception);
+                }
+            });
+    }
+
+    @Test
+    public void testSearchInBloomFilter()
+    {
+        qt().forAll(arbitrary().enumValues(Partitioner.class))
+            .checkAssert(partitioner -> {
+                try (TemporaryDirectory directory = new TemporaryDirectory())
+                {
+                    // Write an SSTable
+                    TestSchema schema = TestSchema.basic(BRIDGE);
+                    schema.writeSSTable(directory, BRIDGE, partitioner, writer -> {
+                        for (int row = 0; row < ROWS; row++)
+                        {
+                            for (int column = 0; column < COLUMNS; column++)
+                            {
+                                writer.write(row, column, row + column);
+                            }
+                        }
+                    });
+                    assertEquals(1, TestSSTable.countIn(directory.path()));
+
+                    ByteBuffer key1 = Int32Type.instance.fromString("1");
+                    BigInteger token1 = BRIDGE.hash(partitioner, key1);
+                    PartitionKeyFilter keyInSSTable = PartitionKeyFilter.create(key1, token1);
+
+                    // Read Filter.db file
+                    Path filterFile = TestSSTable.firstIn(directory.path(), FileType.FILTER);
+                    Descriptor descriptor = Descriptor.fromFilename(filterFile.toFile());
+                    IPartitioner iPartitioner;
+                    switch (partitioner)
+                    {
+                        case Murmur3Partitioner:
+                            iPartitioner = Murmur3Partitioner.instance;
+                            break;
+                        case RandomPartitioner:
+                            iPartitioner = RandomPartitioner.instance;
+                            break;
+                        default:
+                            throw new RuntimeException("Unexpected partitioner: " + partitioner);
+                    }
+
+                    try (InputStream indexStream = new FileInputStream(filterFile.toString()))
+                    {
+                        SSTable ssTable = mock(SSTable.class);
+                        when(ssTable.openFilterStream()).thenReturn(indexStream);
+                        List<PartitionKeyFilter> filters = ReaderUtils.filterKeyInBloomFilter(ssTable,
+                                                                                              iPartitioner,
+                                                                                              descriptor,
+                                                                                              Collections.singletonList(keyInSSTable));
+                        assertEquals(1, filters.size());
+                        assertEquals(keyInSSTable, filters.get(0));
+                    }
+                }
+                catch (IOException exception)
+                {
+                    throw new RuntimeException(exception);
+                }
+            });
+    }
+
+    @Test
+    public void testSearchInIndexEmptyFilters()
+    {
+        qt().forAll(arbitrary().enumValues(Partitioner.class))
+            .checkAssert(partitioner -> {
+                try (TemporaryDirectory directory = new TemporaryDirectory())
+                {
+                    // Write an SSTable
+                    TestSchema schema = TestSchema.basic(BRIDGE);
+                    schema.writeSSTable(directory, BRIDGE, partitioner, writer -> {
+                        for (int row = 0; row < ROWS; row++)
+                        {
+                            for (int column = 0; column < COLUMNS; column++)
+                            {
+                                writer.write(row, column, row + column);
+                            }
+                        }
+                    });
+                    assertEquals(1, TestSSTable.countIn(directory.path()));
+
+                    Path indexFile = TestSSTable.firstIn(directory.path(), FileType.INDEX);
+                    try (InputStream indexStream = new FileInputStream(indexFile.toString()))
+                    {
+                        SSTable ssTable = mock(SSTable.class);
+                        when(ssTable.openPrimaryIndexStream()).thenReturn(indexStream);
+                        assertFalse(ReaderUtils.anyFilterKeyInIndex(ssTable, Collections.emptyList()));
+                    }
+                }
+                catch (IOException exception)
+                {
+                    throw new RuntimeException(exception);
+                }
+            });
+    }
+
+    @Test
+    public void testSearchInIndexKeyNotFound()
+    {
+        qt().forAll(arbitrary().enumValues(Partitioner.class))
+            .checkAssert(partitioner -> {
+                try (TemporaryDirectory directory = new TemporaryDirectory())
+                {
+                    // Write an SSTable
+                    TestSchema schema = TestSchema.basic(BRIDGE);
+                    schema.writeSSTable(directory, BRIDGE, partitioner, writer -> {
+                        for (int row = 0; row < ROWS; row++)
+                        {
+                            for (int column = 0; column < COLUMNS; column++)
+                            {
+                                writer.write(row, column, row + column);
+                            }
+                        }
+                    });
+                    assertEquals(1, TestSSTable.countIn(directory.path()));
+
+                    ByteBuffer key = Int32Type.instance.fromString("51");
+                    BigInteger token = BRIDGE.hash(partitioner, key);
+                    PartitionKeyFilter keyNotInSSTable = PartitionKeyFilter.create(key, token);
+
+                    Path indexFile = TestSSTable.firstIn(directory.path(), FileType.INDEX);
+                    try (InputStream indexStream = new FileInputStream(indexFile.toString()))
+                    {
+                        SSTable ssTable = mock(SSTable.class);
+                        when(ssTable.openPrimaryIndexStream()).thenReturn(indexStream);
+                        assertFalse(ReaderUtils.anyFilterKeyInIndex(ssTable, Collections.singletonList(keyNotInSSTable)));
+                    }
+                }
+                catch (IOException exception)
+                {
+                    throw new RuntimeException(exception);
+                }
+            });
+    }
+
+    @Test
+    public void testSearchInIndexKeyFound()
+    {
+        qt().forAll(arbitrary().enumValues(Partitioner.class))
+            .checkAssert(partitioner -> {
+                try (TemporaryDirectory directory = new TemporaryDirectory())
+                {
+                    // Write an SSTable
+                    TestSchema schema = TestSchema.basic(BRIDGE);
+                    schema.writeSSTable(directory, BRIDGE, partitioner, writer -> {
+                        for (int row = 0; row < ROWS; row++)
+                        {
+                            for (int column = 0; column < COLUMNS; column++)
+                            {
+                                writer.write(row, column, row + column);
+                            }
+                        }
+                    });
+                    assertEquals(1, TestSSTable.countIn(directory.path()));
+
+                    ByteBuffer key = Int32Type.instance.fromString("19");
+                    BigInteger token = BRIDGE.hash(partitioner, key);
+                    PartitionKeyFilter keyInSSTable = PartitionKeyFilter.create(key, token);
+
+                    Path indexFile = TestSSTable.firstIn(directory.path(), FileType.INDEX);
+                    try (InputStream indexStream = new FileInputStream(indexFile.toString()))
+                    {
+                        SSTable ssTable = mock(SSTable.class);
+                        when(ssTable.openPrimaryIndexStream()).thenReturn(indexStream);
+                        assertTrue(ReaderUtils.anyFilterKeyInIndex(ssTable, Collections.singletonList(keyInSSTable)));
+                    }
+                }
+                catch (IOException exception)
+                {
+                    throw new RuntimeException(exception);
+                }
+            });
+    }
+}
diff --git a/cassandra-four-zero/src/test/java/org/apache/cassandra/spark/reader/SSTableCacheTests.java b/cassandra-four-zero/src/test/java/org/apache/cassandra/spark/reader/SSTableCacheTests.java
new file mode 100644
index 0000000..cb4d724
--- /dev/null
+++ b/cassandra-four-zero/src/test/java/org/apache/cassandra/spark/reader/SSTableCacheTests.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.cassandra.spark.reader;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.IntStream;
+
+import com.google.common.collect.ImmutableMap;
+import org.junit.Test;
+
+import org.apache.cassandra.bridge.CassandraBridgeImplementation;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.metadata.MetadataComponent;
+import org.apache.cassandra.io.sstable.metadata.MetadataType;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.spark.data.ReplicationFactor;
+import org.apache.cassandra.spark.data.SSTable;
+import org.apache.cassandra.spark.data.partitioner.Partitioner;
+import org.apache.cassandra.spark.utils.TemporaryDirectory;
+import org.apache.cassandra.spark.utils.test.TestSSTable;
+import org.apache.cassandra.spark.utils.test.TestSchema;
+import org.apache.cassandra.utils.BloomFilter;
+import org.apache.cassandra.utils.Pair;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.quicktheories.QuickTheory.qt;
+import static org.quicktheories.generators.SourceDSL.arbitrary;
+
+public class SSTableCacheTests
+{
+    private static final CassandraBridgeImplementation BRIDGE = new CassandraBridgeImplementation();
+
+    @Test
+    public void testCache()
+    {
+        qt().forAll(arbitrary().enumValues(Partitioner.class))
+            .checkAssert(partitioner -> {
+                try (TemporaryDirectory directory = new TemporaryDirectory())
+                {
+                    // Write an SSTable
+                    TestSchema schema = TestSchema.basic(BRIDGE);
+                    schema.writeSSTable(directory, BRIDGE, partitioner, writer ->
+                            IntStream.range(0, 10).forEach(index -> writer.write(index, 0, index)));
+                    schema.writeSSTable(directory, BRIDGE, partitioner, writer ->
+                            IntStream.range(20, 100).forEach(index -> writer.write(index, 1, index)));
+                    List<SSTable> ssTables = TestSSTable.allIn(directory.path());
+                    String dataFile0 = ssTables.get(0).getDataFileName();
+                    String dataFile1 = ssTables.get(1).getDataFileName();
+                    TableMetadata metadata = new SchemaBuilder(schema.createStatement,
+                                                               schema.keyspace,
+                                                               new ReplicationFactor(ReplicationFactor.ReplicationStrategy.SimpleStrategy,
+                                                                                     ImmutableMap.of("replication_factor", 1)),
+                                                               partitioner).tableMetaData();
+                    SSTable ssTable0 = ssTables.get(0);
+                    assertFalse(SSTableCache.INSTANCE.containsSummary(ssTable0));
+                    assertFalse(SSTableCache.INSTANCE.containsIndex(ssTable0));
+                    assertFalse(SSTableCache.INSTANCE.containsStats(ssTable0));
+
+                    SummaryDbUtils.Summary key1 = SSTableCache.INSTANCE.keysFromSummary(metadata, ssTable0);
+                    assertNotNull(key1);
+                    assertTrue(SSTableCache.INSTANCE.containsSummary(ssTable0));
+                    assertFalse(SSTableCache.INSTANCE.containsIndex(ssTable0));
+                    assertFalse(SSTableCache.INSTANCE.containsStats(ssTable0));
+                    assertFalse(SSTableCache.INSTANCE.containsFilter(ssTable0));
+
+                    Pair<DecoratedKey, DecoratedKey> key2 = SSTableCache.INSTANCE.keysFromIndex(metadata, ssTable0);
+                    assertEquals(key1.first(), key2.left);
+                    assertEquals(key1.last(), key2.right);
+                    assertTrue(SSTableCache.INSTANCE.containsSummary(ssTable0));
+                    assertTrue(SSTableCache.INSTANCE.containsIndex(ssTable0));
+                    assertFalse(SSTableCache.INSTANCE.containsStats(ssTable0));
+                    assertFalse(SSTableCache.INSTANCE.containsFilter(ssTable0));
+
+                    Descriptor descriptor0 = Descriptor.fromFilename(
+                            new File(String.format("./%s/%s", schema.keyspace, schema.table), dataFile0));
+                    Map<MetadataType, MetadataComponent> componentMap = SSTableCache.INSTANCE.componentMapFromStats(ssTable0, descriptor0);
+                    assertNotNull(componentMap);
+                    assertTrue(SSTableCache.INSTANCE.containsSummary(ssTable0));
+                    assertTrue(SSTableCache.INSTANCE.containsIndex(ssTable0));
+                    assertTrue(SSTableCache.INSTANCE.containsStats(ssTable0));
+                    assertFalse(SSTableCache.INSTANCE.containsFilter(ssTable0));
+                    assertEquals(componentMap, SSTableCache.INSTANCE.componentMapFromStats(ssTable0, descriptor0));
+
+                    BloomFilter filter = SSTableCache.INSTANCE.bloomFilter(ssTable0, descriptor0);
+                    assertTrue(SSTableCache.INSTANCE.containsSummary(ssTable0));
+                    assertTrue(SSTableCache.INSTANCE.containsIndex(ssTable0));
+                    assertTrue(SSTableCache.INSTANCE.containsStats(ssTable0));
+                    assertTrue(SSTableCache.INSTANCE.containsFilter(ssTable0));
+                    assertTrue(filter.isPresent(key1.first()));
+                    assertTrue(filter.isPresent(key1.last()));
+
+                    SSTable ssTable1 = ssTables.get(1);
+                    Descriptor descriptor1 = Descriptor.fromFilename(
+                            new File(String.format("./%s/%s", schema.keyspace, schema.table), dataFile1));
+                    assertFalse(SSTableCache.INSTANCE.containsSummary(ssTable1));
+                    assertFalse(SSTableCache.INSTANCE.containsIndex(ssTable1));
+                    assertFalse(SSTableCache.INSTANCE.containsStats(ssTable1));
+                    assertFalse(SSTableCache.INSTANCE.containsFilter(ssTable1));
+                    SummaryDbUtils.Summary key3 = SSTableCache.INSTANCE.keysFromSummary(metadata, ssTable1);
+                    assertNotEquals(key1.first(), key3.first());
+                    assertNotEquals(key1.last(), key3.last());
+                    Pair<DecoratedKey, DecoratedKey> key4 = SSTableCache.INSTANCE.keysFromIndex(metadata, ssTable1);
+                    assertNotEquals(key1.first(), key4.left);
+                    assertNotEquals(key1.last(), key4.right);
+                    assertEquals(SSTableCache.INSTANCE.keysFromSummary(metadata, ssTable1).first(),
+                                 SSTableCache.INSTANCE.keysFromIndex(metadata, ssTable1).left);
+                    assertEquals(SSTableCache.INSTANCE.keysFromSummary(metadata, ssTable1).last(),
+                                 SSTableCache.INSTANCE.keysFromIndex(metadata, ssTable1).right);
+                    assertNotEquals(componentMap, SSTableCache.INSTANCE.componentMapFromStats(ssTable1, descriptor1));
+                    Pair<DecoratedKey, DecoratedKey> key5 = SSTableCache.INSTANCE.keysFromIndex(metadata, ssTable1);
+                    assertTrue(SSTableCache.INSTANCE.bloomFilter(ssTable1, descriptor1).isPresent(key5.left));
+                    assertTrue(SSTableCache.INSTANCE.containsSummary(ssTable1));
+                    assertTrue(SSTableCache.INSTANCE.containsIndex(ssTable1));
+                    assertTrue(SSTableCache.INSTANCE.containsStats(ssTable1));
+                    assertTrue(SSTableCache.INSTANCE.containsFilter(ssTable1));
+                }
+                catch (IOException exception)
+                {
+                    throw new RuntimeException(exception);
+                }
+            });
+    }
+}
diff --git a/cassandra-four-zero/src/test/java/org/apache/cassandra/spark/reader/SSTableReaderTests.java b/cassandra-four-zero/src/test/java/org/apache/cassandra/spark/reader/SSTableReaderTests.java
new file mode 100644
index 0000000..95e9193
--- /dev/null
+++ b/cassandra-four-zero/src/test/java/org/apache/cassandra/spark/reader/SSTableReaderTests.java
@@ -0,0 +1,1162 @@
+/*
+ * 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.cassandra.spark.reader;
+
+import java.io.BufferedInputStream;
+import java.io.DataInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.commons.lang.StringUtils;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.bridge.CassandraBridgeImplementation;
+import org.apache.cassandra.bridge.TokenRange;
+import org.apache.cassandra.db.BufferDecoratedKey;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.db.rows.AbstractRow;
+import org.apache.cassandra.db.rows.Cell;
+import org.apache.cassandra.db.rows.ColumnData;
+import org.apache.cassandra.db.rows.Unfiltered;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.serializers.UTF8Serializer;
+import org.apache.cassandra.spark.data.CqlTable;
+import org.apache.cassandra.spark.data.FileType;
+import org.apache.cassandra.spark.data.ReplicationFactor;
+import org.apache.cassandra.spark.data.SSTable;
+import org.apache.cassandra.spark.data.partitioner.Partitioner;
+import org.apache.cassandra.spark.sparksql.filters.PartitionKeyFilter;
+import org.apache.cassandra.spark.sparksql.filters.SparkRangeFilter;
+import org.apache.cassandra.spark.stats.Stats;
+import org.apache.cassandra.spark.utils.ByteBufferUtils;
+import org.apache.cassandra.spark.utils.TemporaryDirectory;
+import org.apache.cassandra.spark.utils.Throwing;
+import org.apache.cassandra.spark.utils.test.TestSSTable;
+import org.apache.cassandra.spark.utils.test.TestSchema;
+import org.apache.cassandra.utils.Pair;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.quicktheories.QuickTheory.qt;
+import static org.quicktheories.generators.SourceDSL.arbitrary;
+
+public class SSTableReaderTests
+{
+    private static final Logger LOGGER = LoggerFactory.getLogger(SSTableReaderTests.class);
+    private static final CassandraBridgeImplementation BRIDGE = new CassandraBridgeImplementation();
+    private static final int ROWS = 50;
+    private static final int COLUMNS = 25;
+
+    @Test
+    public void testOpenCompressedRawInputStream()
+    {
+        qt().forAll(arbitrary().enumValues(Partitioner.class))
+            .checkAssert(partitioner -> {
+                try (TemporaryDirectory directory = new TemporaryDirectory())
+                {
+                    // Write an SSTable
+                    TestSchema schema = TestSchema.basic(BRIDGE);
+                    schema.writeSSTable(directory, BRIDGE, partitioner, writer -> {
+                        for (int row = 0; row < ROWS; row++)
+                        {
+                            for (int column = 0; column < COLUMNS; column++)
+                            {
+                                writer.write(row, column, row + column);
+                            }
+                        }
+                    });
+                    assertEquals(1, TestSSTable.countIn(directory.path()));
+
+                    // Verify we can open the CompressedRawInputStream and read through the Data.db file
+                    Path dataFile = TestSSTable.firstIn(directory.path(), FileType.DATA);
+                    Descriptor descriptor = Descriptor.fromFilename(
+                            new File(String.format("./%s/%s", schema.keyspace, schema.table), dataFile.getFileName().toString()));
+                    long size = Files.size(dataFile);
+                    assertTrue(size > 0);
+                    Path compressionFile = TestSSTable.firstIn(directory.path(), FileType.COMPRESSION_INFO);
+                    long bytesRead = 0;
+                    try (InputStream dis = new BufferedInputStream(Files.newInputStream(dataFile));
+                         InputStream cis = new BufferedInputStream(Files.newInputStream(compressionFile));
+                         DataInputPlus.DataInputStreamPlus in = new DataInputPlus.DataInputStreamPlus(new DataInputStream(
+                             CompressedRawInputStream.fromInputStream(dis, cis, descriptor.version.hasMaxCompressedLength()))))
+                    {
+                        while (in.read() >= 0)
+                        {
+                            bytesRead++;
+                        }
+                    }
+                    assertTrue(bytesRead > size);
+                }
+                catch (IOException exception)
+                {
+                    throw new RuntimeException(exception);
+                }
+            });
+    }
+
+    @Test
+    public void testOpenSSTableReader()
+    {
+        qt().forAll(arbitrary().enumValues(Partitioner.class))
+            .checkAssert(partitioner -> {
+                try (TemporaryDirectory directory = new TemporaryDirectory())
+                {
+                    // Write an SSTable
+                    TestSchema schema = TestSchema.basic(BRIDGE);
+                    schema.writeSSTable(directory, BRIDGE, partitioner, writer -> {
+                        for (int row = 0; row < ROWS; row++)
+                        {
+                            for (int column = 0; column < COLUMNS; column++)
+                            {
+                                writer.write(row, column, row + column);
+                            }
+                        }
+                    });
+                    assertEquals(1, TestSSTable.countIn(directory.path()));
+
+                    SSTable dataFile = TestSSTable.firstIn(directory.path());
+                    TableMetadata metadata = tableMetadata(schema, partitioner);
+                    SSTableReader reader = openReader(metadata, dataFile);
+
+                    assertNotNull(reader.firstToken());
+                    assertNotNull(reader.lastToken());
+                    assertNotNull(reader.getSSTableMetadata());
+                    assertFalse(reader.isRepaired());
+                    assertEquals(ROWS * COLUMNS, countAndValidateRows(reader));
+                }
+                catch (IOException exception)
+                {
+                    throw new RuntimeException(exception);
+                }
+            });
+    }
+
+    @Test
+    public void testFileNameWithoutPrefix()
+    {
+        qt().forAll(arbitrary().enumValues(Partitioner.class))
+            .checkAssert(partitioner -> {
+                try (TemporaryDirectory directory = new TemporaryDirectory())
+                {
+                    TestSchema schema = TestSchema.basic(BRIDGE);
+                    schema.writeSSTable(directory, BRIDGE, partitioner, writer -> writer.write(42, 43, 44));
+
+                    String prefix = schema.keyspace + "-" + schema.table + "-";
+                    Files.list(directory.path())
+                         .filter(file -> file.getFileName().toString().startsWith(prefix))
+                         .forEach(Throwing.consumer(file -> Files.move(file,
+                             Paths.get(file.getParent().toString(), file.getFileName().toString().replaceFirst("^" + prefix, "")))));
+
+                    TableMetadata metadata = tableMetadata(schema, partitioner);
+                    SSTable table = TestSSTable.firstIn(directory.path());
+                    openReader(metadata, table);
+                }
+                catch (IOException exception)
+                {
+                    throw new RuntimeException(exception);
+                }
+            });
+    }
+
+    @Test
+    public void testFileNameWithPrefix()
+    {
+        qt().forAll(arbitrary().enumValues(Partitioner.class))
+            .checkAssert(partitioner -> {
+                try (TemporaryDirectory directory = new TemporaryDirectory())
+                {
+                    TestSchema schema = TestSchema.basic(BRIDGE);
+                    schema.writeSSTable(directory, BRIDGE, partitioner, writer -> writer.write(42, 43, 44));
+
+                    String prefix = schema.keyspace + "-" + schema.table + "-";
+                    Files.list(directory.path())
+                         .filter(file -> !file.getFileName().toString().startsWith(prefix))
+                         .forEach(Throwing.consumer(file -> Files.move(file,
+                             Paths.get(file.getParent().toString(), prefix + file.getFileName().toString()))));
+
+                    TableMetadata metadata = tableMetadata(schema, partitioner);
+                    SSTable table = TestSSTable.firstIn(directory.path());
+                    openReader(metadata, table);
+                }
+                catch (IOException exception)
+                {
+                    throw new RuntimeException(exception);
+                }
+            });
+    }
+
+    @Test
+    @SuppressWarnings("static-access")
+    public void testSSTableRange()
+    {
+        qt().forAll(arbitrary().enumValues(Partitioner.class))
+            .checkAssert(partitioner -> {
+                try (TemporaryDirectory directory = new TemporaryDirectory())
+                {
+                    // Write an SSTable
+                    TestSchema schema = TestSchema.basic(BRIDGE);
+                    schema.writeSSTable(directory, BRIDGE, partitioner, writer -> {
+                        for (int row = 0; row < 10; row++)
+                        {
+                            for (int column = 0; column < 1; column++)
+                            {
+                                writer.write(row, column, row + column);
+                            }
+                        }
+                    });
+                    assertEquals(1, TestSSTable.countIn(directory.path()));
+
+                    TableMetadata metadata = tableMetadata(schema, partitioner);
+                    SSTable table = TestSSTable.firstIn(directory.path());
+                    SparkSSTableReader reader = openReader(metadata, table);
+                    assertNotNull(reader.firstToken());
+                    assertNotNull(reader.lastToken());
+
+                    // Verify primary Index.db file matches first and last
+                    Path indexFile = TestSSTable.firstIn(directory.path(), FileType.INDEX);
+                    Pair<DecoratedKey, DecoratedKey> firstAndLast;
+                    try (InputStream is = new BufferedInputStream(new FileInputStream(indexFile.toFile())))
+                    {
+                        Pair<ByteBuffer, ByteBuffer> keys = ReaderUtils.readPrimaryIndex(is, true, Collections.emptyList());
+                        firstAndLast = Pair.create(BRIDGE.getPartitioner(partitioner).decorateKey(keys.left),
+                                                   BRIDGE.getPartitioner(partitioner).decorateKey(keys.right));
+                    }
+                    BigInteger first = ReaderUtils.tokenToBigInteger(firstAndLast.left.getToken());
+                    BigInteger last = ReaderUtils.tokenToBigInteger(firstAndLast.right.getToken());
+                    assertEquals(first, reader.firstToken());
+                    assertEquals(last, reader.lastToken());
+
+                    switch (partitioner)
+                    {
+                        case Murmur3Partitioner:
+                            assertFalse(SparkSSTableReader.overlaps(reader,
+                                    TokenRange.closed(Partitioner.Murmur3Partitioner.minToken(),
+                                                      Partitioner.Murmur3Partitioner.minToken())));
+                            assertFalse(SparkSSTableReader.overlaps(reader,
+                                    TokenRange.closed(Partitioner.Murmur3Partitioner.minToken(),
+                                                      Partitioner.Murmur3Partitioner.minToken())));
+                            assertFalse(SparkSSTableReader.overlaps(reader,
+                                    TokenRange.closed(BigInteger.valueOf(-8710962479251732708L),
+                                                      BigInteger.valueOf(-7686143364045646507L))));
+                            assertTrue(SparkSSTableReader.overlaps(reader,
+                                    TokenRange.closed(BigInteger.valueOf(-7509452495886106294L),
+                                                     BigInteger.valueOf(-7509452495886106293L))));
+                            assertTrue(SparkSSTableReader.overlaps(reader,
+                                    TokenRange.closed(BigInteger.valueOf(-7509452495886106293L),
+                                                     BigInteger.valueOf(-7509452495886106293L))));
+                            assertTrue(SparkSSTableReader.overlaps(reader,
+                                    TokenRange.closed(BigInteger.valueOf(-7509452495886106293L),
+                                                     BigInteger.valueOf(2562047788015215502L))));
+                            assertTrue(SparkSSTableReader.overlaps(reader,
+                                    TokenRange.closed(BigInteger.valueOf(-7509452495886106293L),
+                                                     BigInteger.valueOf(9010454139840013625L))));
+                            assertTrue(SparkSSTableReader.overlaps(reader,
+                                    TokenRange.closed(BigInteger.valueOf(9010454139840013625L),
+                                                     BigInteger.valueOf(9010454139840013625L))));
+                            assertFalse(SparkSSTableReader.overlaps(reader,
+                                    TokenRange.closed(Partitioner.Murmur3Partitioner.maxToken(),
+                                                      Partitioner.Murmur3Partitioner.maxToken())));
+                            return;
+                        case RandomPartitioner:
+                            assertFalse(SparkSSTableReader.overlaps(reader,
+                                    TokenRange.closed(Partitioner.RandomPartitioner.minToken(),
+                                                      Partitioner.RandomPartitioner.minToken())));
+                            assertFalse(SparkSSTableReader.overlaps(reader,
+                                    TokenRange.closed(BigInteger.valueOf(0L),
+                                                      BigInteger.valueOf(500L))));
+                            assertFalse(SparkSSTableReader.overlaps(reader,
+                                    TokenRange.closed(new BigInteger("18837662806270881894834867523173387677"),
+                                                      new BigInteger("18837662806270881894834867523173387677"))));
+                            assertTrue(SparkSSTableReader.overlaps(reader,
+                                    TokenRange.closed(new BigInteger("18837662806270881894834867523173387678"),
+                                                      new BigInteger("18837662806270881894834867523173387678"))));
+                            assertTrue(SparkSSTableReader.overlaps(reader,
+                                    TokenRange.closed(new BigInteger("18837662806270881894834867523173387679"),
+                                                      new BigInteger("18837662806270881894834867523173387679"))));
+                            assertTrue(SparkSSTableReader.overlaps(reader,
+                                    TokenRange.closed(new BigInteger("18837662806270881894834867523173387679"),
+                                                      new BigInteger("137731376325982006772573399291321493164"))));
+                            assertTrue(SparkSSTableReader.overlaps(reader,
+                                    TokenRange.closed(new BigInteger("137731376325982006772573399291321493164"),
+                                                      new BigInteger("137731376325982006772573399291321493164"))));
+                            assertFalse(SparkSSTableReader.overlaps(reader,
+                                    TokenRange.closed(new BigInteger("137731376325982006772573399291321493165"),
+                                                      new BigInteger("137731376325982006772573399291321493165"))));
+                            assertFalse(SparkSSTableReader.overlaps(reader,
+                                    TokenRange.closed(Partitioner.RandomPartitioner.maxToken(),
+                                                      Partitioner.RandomPartitioner.maxToken())));
+                            return;
+                        default:
+                            throw new RuntimeException("Unexpected partitioner: " + partitioner);
+                    }
+                }
+                catch (IOException exception)
+                {
+                    throw new RuntimeException(exception);
+                }
+            });
+    }
+
+    @Test
+    public void testSkipNoPartitions()
+    {
+        qt().forAll(arbitrary().enumValues(Partitioner.class))
+            .checkAssert(partitioner -> {
+                try (TemporaryDirectory directory = new TemporaryDirectory())
+                {
+                    // Write an SSTable
+                    TestSchema schema = TestSchema.basic(BRIDGE);
+                    schema.writeSSTable(directory, BRIDGE, partitioner, writer -> {
+                        for (int row = 0; row < ROWS; row++)
+                        {
+                            for (int column = 0; column < COLUMNS; column++)
+                            {
+                                writer.write(row, column, row + column);
+                            }
+                        }
+                    });
+                    assertEquals(1, TestSSTable.countIn(directory.path()));
+
+                    SSTable dataFile = TestSSTable.firstIn(directory.path());
+                    Path summaryFile = TestSSTable.firstIn(directory.path(), FileType.SUMMARY);
+                    TableMetadata metadata = tableMetadata(schema, partitioner);
+                    SummaryDbUtils.Summary summary;
+                    try (InputStream in = new BufferedInputStream(Files.newInputStream(summaryFile)))
+                    {
+                        summary = SummaryDbUtils.readSummary(in,
+                                                             metadata.partitioner,
+                                                             metadata.params.minIndexInterval,
+                                                             metadata.params.maxIndexInterval);
+                    }
+                    // Set Spark token range equal to SSTable token range
+                    TokenRange sparkTokenRange = TokenRange.closed(ReaderUtils.tokenToBigInteger(summary.first().getToken()),
+                                                                   ReaderUtils.tokenToBigInteger(summary.last().getToken()));
+                    SparkRangeFilter rangeFilter = SparkRangeFilter.create(sparkTokenRange);
+                    AtomicBoolean skipped = new AtomicBoolean(false);
+                    Stats stats = new Stats()
+                    {
+                        @Override
+                        public void skippedPartition(ByteBuffer key, BigInteger token)
+                        {
+                            LOGGER.error("Skipped partition when should not: " + token);
+                            skipped.set(true);
+                        }
+                    };
+                    SSTableReader reader = openReader(metadata, dataFile, rangeFilter, true, stats);
+                    assertEquals(ROWS * COLUMNS, countAndValidateRows(reader));  // Shouldn't skip any partitions here
+                    assertFalse(skipped.get());
+                }
+                catch (IOException exception)
+                {
+                    throw new RuntimeException(exception);
+                }
+            });
+    }
+
+    @Test
+    public void testSkipPartitions()
+    {
+        qt().forAll(arbitrary().enumValues(Partitioner.class))
+            .checkAssert(partitioner -> {
+                try (TemporaryDirectory directory = new TemporaryDirectory())
+                {
+                    // Write an SSTable
+                    TestSchema schema = TestSchema.basic(BRIDGE);
+                    schema.writeSSTable(directory, BRIDGE, partitioner, writer -> {
+                        for (int row = 0; row < ROWS; row++)
+                        {
+                            for (int column = 0; column < COLUMNS; column++)
+                            {
+                                writer.write(row, column, row + column);
+                            }
+                        }
+                    });
+                    assertEquals(1, TestSSTable.countIn(directory.path()));
+
+                    SSTable dataFile = TestSSTable.firstIn(directory.path());
+                    TableMetadata metadata = tableMetadata(schema, partitioner);
+                    TokenRange sparkTokenRange;
+                    switch (partitioner)
+                    {
+                        case Murmur3Partitioner:
+                            sparkTokenRange = TokenRange.closed(BigInteger.valueOf(-9223372036854775808L),
+                                                                BigInteger.valueOf(3074457345618258602L));
+                            break;
+                        case RandomPartitioner:
+                            sparkTokenRange = TokenRange.closed(BigInteger.ZERO,
+                                                                new BigInteger("916176208424801638531839357843455255"));
+                            break;
+                        default:
+                            throw new RuntimeException("Unexpected partitioner: " + partitioner);
+                    }
+                    SparkRangeFilter rangeFilter = SparkRangeFilter.create(sparkTokenRange);
+                    AtomicInteger skipCount = new AtomicInteger(0);
+                    AtomicBoolean pass = new AtomicBoolean(true);
+                    Stats stats = new Stats()
+                    {
+                        @Override
+                        public void skippedPartition(ByteBuffer key, BigInteger token)
+                        {
+                            LOGGER.info("Skipping partition: " + token);
+                            skipCount.incrementAndGet();
+                            if (sparkTokenRange.contains(token))
+                            {
+                                LOGGER.info("Should not skip partition: " + token);
+                                pass.set(false);
+                            }
+                        }
+                    };
+                    SSTableReader reader = openReader(metadata, dataFile, rangeFilter, false, stats);
+                    int rows = countAndValidateRows(reader);
+                    assertTrue(skipCount.get() > 0);
+                    assertEquals((ROWS - skipCount.get()) * COLUMNS, rows);  // Should skip out of range partitions here
+                    assertTrue(pass.get());
+                }
+                catch (IOException exception)
+                {
+                    throw new RuntimeException(exception);
+                }
+            });
+    }
+
+    @Test
+    public void testOpenCompactionScanner()
+    {
+        qt().forAll(arbitrary().enumValues(Partitioner.class))
+            .checkAssert(partitioner -> {
+                try (TemporaryDirectory directory = new TemporaryDirectory())
+                {
+                    // Write 3 SSTables
+                    TestSchema schema = TestSchema.basic(BRIDGE);
+                    schema.writeSSTable(directory, BRIDGE, partitioner, writer -> {
+                        for (int row = 0; row < ROWS; row++)
+                        {
+                            for (int column = 0; column < COLUMNS; column++)
+                            {
+                                writer.write(row, column, -1);
+                            }
+                        }
+                    });
+                    schema.writeSSTable(directory, BRIDGE, partitioner, writer -> {
+                        for (int row = 0; row < ROWS; row++)
+                        {
+                            for (int column = 0; column < COLUMNS; column++)
+                            {
+                                writer.write(row, column, -2);
+                            }
+                        }
+                    });
+                    schema.writeSSTable(directory, BRIDGE, partitioner, writer -> {
+                        for (int row = 0; row < ROWS; row++)
+                        {
+                            for (int column = 0; column < COLUMNS; column++)
+                            {
+                                writer.write(row, column, row + column);
+                            }
+                        }
+                    });
+                    assertEquals(3, TestSSTable.countIn(directory.path()));
+
+                    // Open CompactionStreamScanner over 3 SSTables
+                    TableMetadata metadata = tableMetadata(schema, partitioner);
+                    Set<SSTableReader> toCompact = TestSSTable.allIn(directory.path()).stream()
+                            .map(Throwing.function(table -> openReader(metadata, table)))
+                            .collect(Collectors.toSet());
+
+                    int count = 0;
+                    try (CompactionStreamScanner scanner = new CompactionStreamScanner(metadata, partitioner, toCompact))
+                    {
+                        // Iterate through CompactionStreamScanner verifying it correctly compacts data together
+                        Rid rid = scanner.rid();
+                        while (scanner.hasNext())
+                        {
+                            scanner.advanceToNextColumn();
+
+                            // Extract partition key value
+                            int a = rid.getPartitionKey().asIntBuffer().get();
+
+                            // Extract clustering key value and column name
+                            ByteBuffer colBuf = rid.getColumnName();
+                            ByteBuffer clusteringKey = ByteBufferUtils.readBytesWithShortLength(colBuf);
+                            colBuf.get();
+                            String colName = ByteBufferUtils.string(ByteBufferUtils.readBytesWithShortLength(colBuf));
+                            colBuf.get();
+                            if (StringUtils.isEmpty(colName))
+                            {
+                                continue;
+                            }
+                            assertEquals("c", colName);
+                            int b = clusteringKey.asIntBuffer().get();
+
+                            // Extract value column
+                            int c = rid.getValue().asIntBuffer().get();
+
+                            // Verify CompactionIterator compacts 3 SSTables to use last values written
+                            assertEquals(c, a + b);
+                            count++;
+                        }
+                    }
+                    assertEquals(ROWS * COLUMNS, count);
+                }
+                catch (IOException exception)
+                {
+                    throw new RuntimeException(exception);
+                }
+            });
+    }
+
+    @Test
+    public void testFiltersDoNotMatch()
+    {
+        qt().forAll(arbitrary().enumValues(Partitioner.class))
+            .checkAssert(partitioner -> {
+                try (TemporaryDirectory directory = new TemporaryDirectory())
+                {
+                    // Write an SSTable
+                    TestSchema schema = TestSchema.basic(BRIDGE);
+                    schema.writeSSTable(directory, BRIDGE, partitioner, writer -> {
+                        for (int row = 0; row < ROWS; row++)
+                        {
+                            for (int column = 0; column < COLUMNS; column++)
+                            {
+                                writer.write(row, column, row + column);
+                            }
+                        }
+                    });
+                    assertEquals(1, TestSSTable.countIn(directory.path()));
+
+                    SSTable dataFile = TestSSTable.firstIn(directory.path());
+                    TableMetadata metadata = tableMetadata(schema, partitioner);
+
+                    BigInteger token = BigInteger.valueOf(9010454139840013626L);
+                    SparkRangeFilter outsideRange = SparkRangeFilter.create(TokenRange.singleton(token));
+
+                    AtomicBoolean pass = new AtomicBoolean(true);
+                    AtomicInteger skipCount = new AtomicInteger(0);
+                    Stats stats = new Stats()
+                    {
+                        @Override
+                        public void skippedSSTable(@Nullable SparkRangeFilter sparkRangeFilter,
+                                                   @NotNull List<PartitionKeyFilter> partitionKeyFilters,
+                                                   @NotNull BigInteger firstToken,
+                                                   @NotNull BigInteger lastToken)
+                        {
+                            skipCount.incrementAndGet();
+                            if (sparkRangeFilter == null || partitionKeyFilters.size() != 0)
+                            {
+                                pass.set(false);
+                            }
+                        }
+                    };
+                    SSTableReader reader = openReader(metadata, dataFile, outsideRange, true, stats);
+                    assertTrue(reader.ignore());
+                    assertEquals(1, skipCount.get());
+                    assertTrue(pass.get());
+                }
+                catch (IOException exception)
+                {
+                    throw new RuntimeException(exception);
+                }
+            });
+    }
+
+    @Test
+    public void testFilterKeyMissingInIndex()
+    {
+        qt().forAll(arbitrary().enumValues(Partitioner.class))
+            .checkAssert(partitioner -> {
+                try (TemporaryDirectory directory = new TemporaryDirectory())
+                {
+                    // Write an SSTable
+                    TestSchema schema = TestSchema.basic(BRIDGE);
+                    schema.writeSSTable(directory, BRIDGE, partitioner, writer -> {
+                        for (int row = 0; row < ROWS; row++)
+                        {
+                            for (int column = 0; column < COLUMNS; column++)
+                            {
+                                writer.write(row, column, row + column);
+                            }
+                        }
+                    });
+                    assertEquals(1, TestSSTable.countIn(directory.path()));
+
+                    SSTable dataFile = TestSSTable.firstIn(directory.path());
+                    TableMetadata metadata = tableMetadata(schema, partitioner);
+
+                    ByteBuffer key1 = Int32Type.instance.fromString("51");
+                    BigInteger token1 = BRIDGE.hash(partitioner, key1);
+                    PartitionKeyFilter keyNotInSSTable1 = PartitionKeyFilter.create(key1, token1);
+                    ByteBuffer key2 = Int32Type.instance.fromString("90");
+                    BigInteger token2 = BRIDGE.hash(partitioner, key2);
+                    PartitionKeyFilter keyNotInSSTable2 = PartitionKeyFilter.create(key2, token2);
+                    List<PartitionKeyFilter> partitionKeyFilters = Arrays.asList(keyNotInSSTable1, keyNotInSSTable2);
+
+                    AtomicBoolean pass = new AtomicBoolean(true);
+                    AtomicInteger skipCount = new AtomicInteger(0);
+                    Stats stats = new Stats()
+                    {
+                        @Override
+                        public void skippedSSTable(@Nullable SparkRangeFilter sparkRangeFilter,
+                                                   @NotNull List<PartitionKeyFilter> partitionKeyFilters,
+                                                   @NotNull BigInteger firstToken,
+                                                   @NotNull BigInteger lastToken)
+                        {
+                            pass.set(false);
+                        }
+
+                        @Override
+                        public void missingInIndex()
+                        {
+                            skipCount.incrementAndGet();
+                            if (partitionKeyFilters.size() != 2)
+                            {
+                                pass.set(false);
+                            }
+                        }
+                    };
+                    SSTableReader reader = openReader(metadata, dataFile, partitionKeyFilters, true, stats);
+                    assertTrue(reader.ignore());
+                    assertEquals(1, skipCount.get());
+                    assertTrue(pass.get());
+                }
+                catch (IOException exception)
+                {
+                    throw new RuntimeException(exception);
+                }
+            });
+    }
+
+    @Test
+    public void testPartialFilterMatch()
+    {
+        qt().forAll(arbitrary().enumValues(Partitioner.class))
+            .checkAssert(partitioner -> {
+                try (TemporaryDirectory directory = new TemporaryDirectory())
+                {
+                    // Write an SSTable
+                    TestSchema schema = TestSchema.basic(BRIDGE);
+                    schema.writeSSTable(directory, BRIDGE, partitioner, writer -> {
+                        for (int row = 0; row < ROWS; row++)
+                        {
+                            for (int column = 0; column < COLUMNS; column++)
+                            {
+                                writer.write(row, column, row + column);
+                            }
+                        }
+                    });
+                    assertEquals(1, TestSSTable.countIn(directory.path()));
+
+                    SSTable dataFile = TestSSTable.firstIn(directory.path());
+                    TableMetadata metadata = tableMetadata(schema, partitioner);
+
+                    ByteBuffer key1 = Int32Type.instance.fromString("0");
+                    BigInteger token1 = BRIDGE.hash(partitioner, key1);
+                    PartitionKeyFilter keyInSSTable = PartitionKeyFilter.create(key1, token1);
+                    SparkRangeFilter rangeFilter = SparkRangeFilter.create(TokenRange.singleton(token1));
+
+                    ByteBuffer key2 = Int32Type.instance.fromString("55");
+                    BigInteger token2 = BRIDGE.hash(partitioner, key2);
+                    PartitionKeyFilter keyNotInSSTable = PartitionKeyFilter.create(key2, token2);
+                    List<PartitionKeyFilter> partitionKeyFilters = Arrays.asList(keyInSSTable, keyNotInSSTable);
+
+                    AtomicBoolean pass = new AtomicBoolean(true);
+                    AtomicInteger skipCount = new AtomicInteger(0);
+                    Stats stats = new Stats()
+                    {
+                        @Override
+                        public void skippedPartition(ByteBuffer key, BigInteger token)
+                        {
+                            LOGGER.info("Skipping partition: " + token);
+                            skipCount.incrementAndGet();
+                            if (partitionKeyFilters.stream().anyMatch(filter -> filter.matches(key)))
+                            {
+                                LOGGER.info("Should not skip partition: " + token);
+                                pass.set(false);
+                            }
+                        }
+                    };
+                    SSTableReader reader = openReader(metadata, dataFile, rangeFilter, partitionKeyFilters, false, stats);
+                    int rows = countAndValidateRows(reader);
+                    assertTrue(skipCount.get() > 0);
+                    assertEquals(COLUMNS, rows);
+                    assertEquals((ROWS - skipCount.get()) * COLUMNS, rows);  // Should skip partitions not matching filters
+                    assertTrue(pass.get());
+                }
+                catch (IOException exception)
+                {
+                    throw new RuntimeException(exception);
+                }
+            });
+    }
+
+    @Test
+    public void testConstructFilename()
+    {
+        // Standard SSTable data file name
+        assertEquals(new File("./keyspace/table/na-1-big-Data.db"),
+                              SSTableReader.constructFilename("keyspace", "table", "na-1-big-Data.db"));
+
+        // Non-standard SSTable data file name
+        assertEquals(new File("./keyspace/table/na-1-big-Data.db"),
+                              SSTableReader.constructFilename("keyspace", "table", "keyspace-table-na-1-big-Data.db"));
+
+        // Malformed SSTable data file names
+        assertEquals(new File("./keyspace/table/keyspace-table-qwerty-na-1-big-Data.db"),
+                              SSTableReader.constructFilename("keyspace", "table", "keyspace-table-qwerty-na-1-big-Data.db"));
+        assertEquals(new File("./keyspace/table/keyspace-qwerty-na-1-big-Data.db"),
+                              SSTableReader.constructFilename("keyspace", "table", "keyspace-qwerty-na-1-big-Data.db"));
+        assertEquals(new File("./keyspace/table/qwerty-table-na-1-big-Data.db"),
+                              SSTableReader.constructFilename("keyspace", "table", "qwerty-table-na-1-big-Data.db"));
+        assertEquals(new File("./keyspace/table/keyspace-na-1-big-Data.db"),
+                              SSTableReader.constructFilename("keyspace", "table", "keyspace-na-1-big-Data.db"));
+        assertEquals(new File("./keyspace/table/table-na-1-big-Data.db"),
+                              SSTableReader.constructFilename("keyspace", "table", "table-na-1-big-Data.db"));
+        assertEquals(new File("./keyspace/table/qwerty.db"),
+                              SSTableReader.constructFilename("keyspace", "table", "qwerty.db"));
+    }
+
+    @Test
+    public void testExtractRangeSparkFilter()
+    {
+        Optional<TokenRange> range1 = SSTableReader.extractRange(
+                SparkRangeFilter.create(TokenRange.closed(BigInteger.valueOf(5L), BigInteger.valueOf(500L))),
+                Collections.emptyList());
+        assertTrue(range1.isPresent());
+        assertEquals(BigInteger.valueOf(5L), range1.get().lowerEndpoint());
+        assertEquals(BigInteger.valueOf(500L), range1.get().upperEndpoint());
+
+        Optional<TokenRange> range2 = SSTableReader.extractRange(
+                SparkRangeFilter.create(TokenRange.closed(BigInteger.valueOf(-10000L), BigInteger.valueOf(29593L))),
+                Collections.emptyList());
+        assertTrue(range2.isPresent());
+        assertEquals(BigInteger.valueOf(-10000L), range2.get().lowerEndpoint());
+        assertEquals(BigInteger.valueOf(29593L), range2.get().upperEndpoint());
+
+        assertFalse(SSTableReader.extractRange(null, Collections.emptyList()).isPresent());
+    }
+
+    @Test
+    public void testExtractRangePartitionKeyFilters()
+    {
+        List<ByteBuffer> keys = new ArrayList<>();
+        for (int index = 0; index < 1000; index++)
+        {
+            // Cast to ByteBuffer required when compiling with Java 8
+            keys.add((ByteBuffer) ByteBuffer.allocate(4).putInt(index).flip());
+        }
+
+        List<PartitionKeyFilter> partitionKeyFilters = keys.stream().map(buffer -> {
+            BigInteger token = ReaderUtils.tokenToBigInteger(Murmur3Partitioner.instance.getToken(buffer).getToken());
+            return PartitionKeyFilter.create(buffer, token);
+        }).collect(Collectors.toList());
+
+        TokenRange sparkRange = TokenRange.closed(new BigInteger("0"), new BigInteger("2305843009213693952"));
+        SparkRangeFilter sparkRangeFilter = SparkRangeFilter.create(sparkRange);
+        List<PartitionKeyFilter> inRangePartitionKeyFilters = partitionKeyFilters.stream()
+                .filter(filter -> sparkRange.contains(filter.token()))
+                .collect(Collectors.toList());
+        assertTrue(inRangePartitionKeyFilters.size() > 1);
+
+        Optional<TokenRange> range = SSTableReader.extractRange(sparkRangeFilter, inRangePartitionKeyFilters);
+        assertTrue(range.isPresent());
+        assertNotEquals(sparkRange, range.get());
+        assertTrue(sparkRange.lowerEndpoint().compareTo(range.get().lowerEndpoint()) < 0);
+        assertTrue(sparkRange.upperEndpoint().compareTo(range.get().upperEndpoint()) > 0);
+    }
+
+    // Incremental Repair
+
+    @Test
+    public void testIncrementalRepair()
+    {
+        qt().forAll(arbitrary().enumValues(Partitioner.class))
+            .checkAssert(partitioner -> {
+                try (TemporaryDirectory directory = new TemporaryDirectory())
+                {
+                    TestSchema schema = TestSchema.basic(BRIDGE);
+                    int numSSTables = 4;
+                    int numRepaired = 2;
+                    int numUnRepaired = numSSTables - numRepaired;
+
+                    // Write some SSTables
+                    for (int table = 0; table < numSSTables; table++)
+                    {
+                        int position = table * ROWS;
+                        schema.writeSSTable(directory, BRIDGE, partitioner, writer -> {
+                            for (int row = position; row < position + ROWS; row++)
+                            {
+                                for (int column = 0; column < COLUMNS; column++)
+                                {
+                                    writer.write(row, column, row + column);
+                                }
+                            }
+                        });
+                    }
+                    assertEquals(numSSTables, TestSSTable.countIn(directory.path()));
+
+                    TableMetadata metadata = tableMetadata(schema, partitioner);
+
+                    AtomicInteger skipCount = new AtomicInteger(0);
+                    Stats stats = new Stats()
+                    {
+                        @Override
+                        public void skippedRepairedSSTable(SSTable ssTable, long repairedAt)
+                        {
+                            skipCount.incrementAndGet();
+                        }
+                    };
+
+                    // Mark some SSTables as repaired
+                    Map<SSTable, Boolean> isRepaired = TestSSTable.allIn(directory.path()).stream()
+                            .collect(Collectors.toMap(Function.identity(), ssTable -> false));
+                    int count = 0;
+                    for (SSTable ssTable : isRepaired.keySet())
+                    {
+                        if (count < numRepaired)
+                        {
+                            isRepaired.put(ssTable, true);
+                            count++;
+                        }
+                    }
+
+                    List<SSTableReader> primaryReaders = TestSSTable.allIn(directory.path()).stream()
+                            .map(ssTable -> openIncrementalReader(metadata, ssTable, stats, true, isRepaired.get(ssTable)))
+                            .filter(reader -> !reader.ignore())
+                            .collect(Collectors.toList());
+                    List<SSTableReader> nonPrimaryReaders = TestSSTable.allIn(directory.path()).stream()
+                            .map(ssTable -> openIncrementalReader(metadata, ssTable, stats, false, isRepaired.get(ssTable)))
+                            .filter(reader -> !reader.ignore())
+                            .collect(Collectors.toList());
+
+                    // Primary repair replica should read all SSTables
+                    assertEquals(numSSTables, primaryReaders.size());
+
+                    // Non-primary repair replica should only read unrepaired SSTables
+                    assertEquals(numUnRepaired, nonPrimaryReaders.size());
+                    for (SSTableReader reader : nonPrimaryReaders)
+                    {
+                        assertFalse(isRepaired.get(reader.sstable()));
+                    }
+                    assertEquals(numUnRepaired, skipCount.get());
+
+                    Set<SSTableReader> toCompact = Stream.concat(
+                            primaryReaders.stream().filter(reader -> isRepaired.get(reader.sstable())),
+                            nonPrimaryReaders.stream()).collect(Collectors.toSet());
+                    assertEquals(numSSTables, toCompact.size());
+
+                    int rowCount = 0;
+                    boolean[] found = new boolean[numSSTables * ROWS];
+                    try (CompactionStreamScanner scanner = new CompactionStreamScanner(metadata, partitioner, toCompact))
+                    {
+                        // Iterate through CompactionScanner and verify we have all the partition keys we are looking for
+                        Rid rid = scanner.rid();
+                        while (scanner.hasNext())
+                        {
+                            scanner.advanceToNextColumn();
+                            int a = rid.getPartitionKey().asIntBuffer().get();
+                            found[a] = true;
+                            // Extract clustering key value and column name
+                            ByteBuffer colBuf = rid.getColumnName();
+                            ByteBuffer clusteringKey = ByteBufferUtils.readBytesWithShortLength(colBuf);
+                            colBuf.get();
+                            String colName = ByteBufferUtils.string(ByteBufferUtils.readBytesWithShortLength(colBuf));
+                            colBuf.get();
+                            if (StringUtils.isEmpty(colName))
+                            {
+                                continue;
+                            }
+                            assertEquals("c", colName);
+                            int b = clusteringKey.asIntBuffer().get();
+
+                            // Extract value column
+                            int c = rid.getValue().asIntBuffer().get();
+
+                            assertEquals(c, a + b);
+                            rowCount++;
+                        }
+                    }
+                    assertEquals(numSSTables * ROWS * COLUMNS, rowCount);
+                    for (boolean b : found)
+                    {
+                        assertTrue(b);
+                    }
+                }
+                catch (IOException exception)
+                {
+                    throw new RuntimeException(exception);
+                }
+            });
+    }
+
+    @Test
+    public void testPartitionKeyFilter()
+    {
+        qt().forAll(arbitrary().enumValues(Partitioner.class))
+            .checkAssert(partitioner -> {
+                try (TemporaryDirectory directory = new TemporaryDirectory())
+                {
+                    TestSchema schema = TestSchema.builder()
+                                                  .withPartitionKey("a", BRIDGE.text())
+                                                  .withClusteringKey("b", BRIDGE.aInt())
+                                                  .withColumn("c", BRIDGE.aInt())
+                                                  .withColumn("d", BRIDGE.text())
+                                                  .build();
+                    CqlTable cqlTable = schema.buildTable();
+                    int numSSTables = 24;
+                    String partitionKeyStr = (String) BRIDGE.text().randomValue(1024);
+                    AbstractMap.SimpleEntry<ByteBuffer, BigInteger> partitionKey =
+                            BRIDGE.getPartitionKey(cqlTable, partitioner, Collections.singletonList(partitionKeyStr));
+                    PartitionKeyFilter partitionKeyFilter = PartitionKeyFilter.create(partitionKey.getKey(),
+                                                                                      partitionKey.getValue());
+                    SparkRangeFilter sparkRangeFilter = SparkRangeFilter.create(TokenRange.closed(partitioner.minToken(),
+                                                                                                  partitioner.maxToken()));
+                    Integer[] expectedC = new Integer[COLUMNS];
+                    String[] expectedD = new String[COLUMNS];
+
+                    // Write some SSTables
+                    for (int table = 0; table < numSSTables; table++)
+                    {
+                        boolean isLastSSTable = table == numSSTables - 1;
+                        schema.writeSSTable(directory, BRIDGE, partitioner, writer -> {
+                            if (isLastSSTable)
+                            {
+                                // Write partition key in last SSTable only
+                                for (int column = 0; column < COLUMNS; column++)
+                                {
+                                    expectedC[column] = (int) BRIDGE.aInt().randomValue(1024);
+                                    expectedD[column] = (String) BRIDGE.text().randomValue(1024);
+                                    writer.write(partitionKeyStr, column, expectedC[column], expectedD[column]);
+                                }
+                            }
+
+                            for (int row = 0; row < 2; row++)
+                            {
+                                for (int column = 0; column < COLUMNS; column++)
+                                {
+                                    String key = null;
+                                    while (key == null || key.equals(partitionKeyStr))
+                                    {
+                                        key = (String) BRIDGE.text().randomValue(1024);
+                                    }
+                                    writer.write(key,
+                                                 row,
+                                                 BRIDGE.aInt().randomValue(1024),
+                                                 BRIDGE.text().randomValue(1024));
+                                }
+                            }
+                        });
+                    }
+
+                    TableMetadata metadata = new SchemaBuilder(schema.createStatement,
+                                                               schema.keyspace,
+                                                               new ReplicationFactor(ReplicationFactor.ReplicationStrategy.SimpleStrategy,
+                                                                                     ImmutableMap.of("replication_factor", 1)),
+                                                               partitioner).tableMetaData();
+                    List<SSTable> ssTables = TestSSTable.allIn(directory.path());
+                    assertEquals(numSSTables, ssTables.size());
+
+                    Set<String> keys = new HashSet<>();
+                    for (SSTable ssTable : ssTables)
+                    {
+                        SSTableReader reader = readerBuilder(metadata, ssTable, Stats.DoNothingStats.INSTANCE, true, false)
+                                .withPartitionKeyFilter(partitionKeyFilter)
+                                .withSparkRangeFilter(sparkRangeFilter)
+                                .build();
+                        if (reader.ignore())
+                        {
+                            continue;
+                        }
+
+                        ISSTableScanner scanner = reader.scanner();
+                        int colCount = 0;
+                        while (scanner.hasNext())
+                        {
+                            UnfilteredRowIterator it = scanner.next();
+                            it.partitionKey().getKey().mark();
+                            String key = UTF8Serializer.instance.deserialize(it.partitionKey().getKey());
+                            it.partitionKey().getKey().reset();
+                            keys.add(key);
+                            while (it.hasNext())
+                            {
+                                it.next();
+                                colCount++;
+                            }
+                        }
+                        assertEquals(COLUMNS, colCount);
+                    }
+                    assertEquals(1, keys.size());
+                    assertEquals(partitionKeyStr, keys.stream()
+                                                      .findFirst()
+                                                      .orElseThrow(() -> new RuntimeException("No partition keys returned")));
+                }
+                catch (IOException exception)
+                {
+                    throw new RuntimeException(exception);
+                }
+            });
+    }
+
+    private static TableMetadata tableMetadata(TestSchema schema, Partitioner partitioner)
+    {
+        return new SchemaBuilder(schema.createStatement,
+                                 schema.keyspace,
+                                 new ReplicationFactor(ReplicationFactor.ReplicationStrategy.SimpleStrategy,
+                                                       ImmutableMap.of("replication_factor", 1)),
+                                 partitioner).tableMetaData();
+    }
+
+    private static SSTableReader openReader(TableMetadata metadata, SSTable ssTable) throws IOException
+    {
+        return openReader(metadata, ssTable, null, Collections.emptyList(), true, Stats.DoNothingStats.INSTANCE);
+    }
+
+    private static SSTableReader openReader(TableMetadata metadata,
+                                            SSTable ssTable,
+                                            SparkRangeFilter sparkRangeFilter) throws IOException
+    {
+        return openReader(metadata, ssTable, sparkRangeFilter, Collections.emptyList(), true, Stats.DoNothingStats.INSTANCE);
+    }
+
+    private static SSTableReader openReader(TableMetadata metadata,
+                                            SSTable ssTable,
+                                            SparkRangeFilter sparkRangeFilter,
+                                            boolean readIndexOffset,
+                                            Stats stats) throws IOException
+    {
+        return openReader(metadata, ssTable, sparkRangeFilter, Collections.emptyList(), readIndexOffset, stats);
+    }
+
+    private static SSTableReader openReader(TableMetadata metadata,
+                                            SSTable ssTable,
+                                            List<PartitionKeyFilter> partitionKeyFilters,
+                                            boolean readIndexOffset,
+                                            Stats stats) throws IOException
+    {
+        return openReader(metadata, ssTable, null, partitionKeyFilters, readIndexOffset, stats);
+    }
+
+    private static SSTableReader openReader(TableMetadata metadata,
+                                            SSTable ssTable,
+                                            SparkRangeFilter sparkRangeFilter,
+                                            List<PartitionKeyFilter> partitionKeyFilters,
+                                            boolean readIndexOffset,
+                                            Stats stats) throws IOException
+    {
+        return SSTableReader.builder(metadata, ssTable)
+                            .withSparkRangeFilter(sparkRangeFilter)
+                            .withPartitionKeyFilters(partitionKeyFilters)
+                            .withReadIndexOffset(readIndexOffset)
+                            .withStats(stats)
+                            .build();
+    }
+
+    private static SSTableReader.Builder readerBuilder(TableMetadata metadata,
+                                                       SSTable ssTable,
+                                                       Stats stats,
+                                                       boolean isRepairPrimary,
+                                                       boolean isRepaired)
+    {
+        return SSTableReader.builder(metadata, ssTable)
+                            .withReadIndexOffset(true)
+                            .withStats(stats)
+                            .isRepairPrimary(isRepairPrimary)
+                            .withIsRepairedFunction(statsMetadata -> isRepaired);
+    }
+
+    private static SSTableReader openIncrementalReader(TableMetadata metadata,
+                                                       SSTable ssTable,
+                                                       Stats stats,
+                                                       boolean isRepairPrimary,
+                                                       boolean isRepaired)
+    {
+        try
+        {
+            return readerBuilder(metadata, ssTable, stats, isRepairPrimary, isRepaired)
+                    .useIncrementalRepair(true)
+                    .build();
+        }
+        catch (IOException exception)
+        {
+            throw new RuntimeException(exception);
+        }
+    }
+
+    private static int countAndValidateRows(@NotNull SSTableReader reader)
+    {
+        ISSTableScanner scanner = reader.scanner();
+        int count = 0;
+        while (scanner.hasNext())
+        {
+            UnfilteredRowIterator it = scanner.next();
+            while (it.hasNext())
+            {
+                BufferDecoratedKey key = (BufferDecoratedKey) it.partitionKey();
+                int a = key.getKey().asIntBuffer().get();
+                Unfiltered unfiltered = it.next();
+                assertTrue(unfiltered.isRow());
+                AbstractRow row = (AbstractRow) unfiltered;
+                int b = row.clustering().bufferAt(0).asIntBuffer().get();
+                for (ColumnData data : row)
+                {
+                    Cell<?> cell = (Cell<?>) data;
+                    int c = cell.buffer().getInt();
+                    assertEquals(c, a + b);
+                    count++;
+                }
+            }
+        }
+        return count;
+    }
+}
diff --git a/cassandra-four-zero/src/test/java/org/apache/cassandra/spark/reader/SchemaBuilderTests.java b/cassandra-four-zero/src/test/java/org/apache/cassandra/spark/reader/SchemaBuilderTests.java
new file mode 100644
index 0000000..275040e
--- /dev/null
+++ b/cassandra-four-zero/src/test/java/org/apache/cassandra/spark/reader/SchemaBuilderTests.java
@@ -0,0 +1,117 @@
+/*
+ * 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.cassandra.spark.reader;
+
+import java.util.HashMap;
+
+import org.junit.Test;
+
+import org.apache.cassandra.bridge.CassandraBridgeImplementation;
+import org.apache.cassandra.cql3.CQLFragmentParser;
+import org.apache.cassandra.cql3.CqlParser;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.schema.KeyspaceMetadata;
+import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.schema.Types;
+import org.apache.cassandra.spark.data.ReplicationFactor;
+import org.apache.cassandra.utils.FBUtilities;
+
+import static org.apache.cassandra.spark.reader.SchemaBuilder.rfToMap;
+import static org.junit.Assert.assertNotNull;
+
+public class SchemaBuilderTests
+{
+    @Test
+    public void getCompactionClass()
+    {
+        FBUtilities.classForName("org.apache.cassandra.db.compaction.LeveledCompactionStrategy", "LeveledCompactionStrategy");
+    }
+
+    @Test
+    public void testDataTypes()
+    {
+        assertNotNull(FBUtilities.classForName("org.apache.cassandra.dht.Murmur3Partitioner", "Murmur3Partitioner"));
+        assertNotNull(FBUtilities.classForName("org.apache.cassandra.dht.RandomPartitioner", "RandomPartitioner"));
+        assertNotNull(FBUtilities.classForName("org.apache.cassandra.db.marshal.AbstractCompositeType", "AbstractCompositeType"));
+        assertNotNull(FBUtilities.classForName("org.apache.cassandra.db.marshal.AbstractType", "AbstractType"));
+        assertNotNull(FBUtilities.classForName("org.apache.cassandra.db.marshal.AsciiType", "AsciiType"));
+        assertNotNull(FBUtilities.classForName("org.apache.cassandra.db.marshal.BooleanType", "BooleanType"));
+        assertNotNull(FBUtilities.classForName("org.apache.cassandra.db.marshal.BytesType", "BytesType"));
+        assertNotNull(FBUtilities.classForName("org.apache.cassandra.db.marshal.ByteType", "ByteType"));
+        assertNotNull(FBUtilities.classForName("org.apache.cassandra.db.marshal.CollectionType", "CollectionType"));
+        assertNotNull(FBUtilities.classForName("org.apache.cassandra.db.marshal.CompositeType", "CompositeType"));
+        assertNotNull(FBUtilities.classForName("org.apache.cassandra.db.marshal.CounterColumnType", "CounterColumnType"));
+        assertNotNull(FBUtilities.classForName("org.apache.cassandra.db.marshal.DateType", "DateType"));
+        assertNotNull(FBUtilities.classForName("org.apache.cassandra.db.marshal.DecimalType", "DecimalType"));
+        assertNotNull(FBUtilities.classForName("org.apache.cassandra.db.marshal.DoubleType", "DoubleType"));
+        assertNotNull(FBUtilities.classForName("org.apache.cassandra.db.marshal.DurationType", "DurationType"));
+        assertNotNull(FBUtilities.classForName("org.apache.cassandra.db.marshal.DynamicCompositeType", "DynamicCompositeType"));
+        assertNotNull(FBUtilities.classForName("org.apache.cassandra.db.marshal.EmptyType", "EmptyType"));
+        assertNotNull(FBUtilities.classForName("org.apache.cassandra.db.marshal.FloatType", "FloatType"));
+        assertNotNull(FBUtilities.classForName("org.apache.cassandra.db.marshal.FrozenType", "FrozenType"));
+        assertNotNull(FBUtilities.classForName("org.apache.cassandra.db.marshal.InetAddressType", "InetAddressType"));
+        assertNotNull(FBUtilities.classForName("org.apache.cassandra.db.marshal.Int32Type", "Int32Type"));
+        assertNotNull(FBUtilities.classForName("org.apache.cassandra.db.marshal.IntegerType", "IntegerType"));
+        assertNotNull(FBUtilities.classForName("org.apache.cassandra.db.marshal.LexicalUUIDType", "LexicalUUIDType"));
+        assertNotNull(FBUtilities.classForName("org.apache.cassandra.db.marshal.ListType", "ListType"));
+        assertNotNull(FBUtilities.classForName("org.apache.cassandra.db.marshal.LongType", "LongType"));
+        assertNotNull(FBUtilities.classForName("org.apache.cassandra.db.marshal.MapType", "MapType"));
+        assertNotNull(FBUtilities.classForName("org.apache.cassandra.db.marshal.NumberType", "NumberType"));
+        assertNotNull(FBUtilities.classForName("org.apache.cassandra.db.marshal.PartitionerDefinedOrder", "PartitionerDefinedOrder"));
+        assertNotNull(FBUtilities.classForName("org.apache.cassandra.db.marshal.ReversedType", "ReversedType"));
+        assertNotNull(FBUtilities.classForName("org.apache.cassandra.db.marshal.SetType", "SetType"));
+        assertNotNull(FBUtilities.classForName("org.apache.cassandra.db.marshal.ShortType", "ShortType"));
+        assertNotNull(FBUtilities.classForName("org.apache.cassandra.db.marshal.SimpleDateType", "SimpleDateType"));
+        assertNotNull(FBUtilities.classForName("org.apache.cassandra.db.marshal.TemporalType", "TemporalType"));
+        assertNotNull(FBUtilities.classForName("org.apache.cassandra.db.marshal.TimestampType", "TimestampType"));
+        assertNotNull(FBUtilities.classForName("org.apache.cassandra.db.marshal.TimeType", "TimeType"));
+        assertNotNull(FBUtilities.classForName("org.apache.cassandra.db.marshal.TimeUUIDType", "TimeUUIDType"));
+        assertNotNull(FBUtilities.classForName("org.apache.cassandra.db.marshal.TupleType", "TupleType"));
+        assertNotNull(FBUtilities.classForName("org.apache.cassandra.db.marshal.TypeParser", "TypeParser"));
+        assertNotNull(FBUtilities.classForName("org.apache.cassandra.db.marshal.UserType", "UserType"));
+        assertNotNull(FBUtilities.classForName("org.apache.cassandra.db.marshal.UTF8Type", "UTF8Type"));
+        assertNotNull(FBUtilities.classForName("org.apache.cassandra.db.marshal.UUIDType", "UUIDType"));
+    }
+
+    @Test
+    public void testSchemaBuilderWithPartiallyInitializedMetadata()
+    {
+        CassandraBridgeImplementation.setup();
+        String keyspaceName = "foo" + getClass().getSimpleName();
+        ReplicationFactor replicationFactor = new ReplicationFactor(ReplicationFactor.ReplicationStrategy.LocalStrategy, new HashMap<>());
+        KeyspaceMetadata keyspaceMetadata = KeyspaceMetadata.create(keyspaceName, KeyspaceParams.create(true, rfToMap(replicationFactor)));
+        Schema.instance.load(keyspaceMetadata);
+        Keyspace.openWithoutSSTables(keyspaceName);
+
+        String createTableStatement = "CREATE TABLE " + keyspaceName + ".bar (a int PRIMARY KEY)";
+        TableMetadata tableMetadata = CQLFragmentParser
+                .parseAny(CqlParser::createTableStatement, createTableStatement, "CREATE TABLE")
+                .keyspace(keyspaceName)
+                .prepare(null)
+                .builder(Types.none())
+                .build();
+        KeyspaceMetadata keyspace = Schema.instance.getKeyspaceMetadata(keyspaceName);
+        Schema.instance.load(keyspace.withSwapped(keyspace.tables.with(tableMetadata)));
+
+        new SchemaBuilder(createTableStatement, keyspaceName, replicationFactor);
+    }
+}
diff --git a/cassandra-four-zero/src/test/java/org/apache/cassandra/spark/reader/SummaryDbTests.java b/cassandra-four-zero/src/test/java/org/apache/cassandra/spark/reader/SummaryDbTests.java
new file mode 100644
index 0000000..f61132c
--- /dev/null
+++ b/cassandra-four-zero/src/test/java/org/apache/cassandra/spark/reader/SummaryDbTests.java
@@ -0,0 +1,176 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.cassandra.spark.reader;
+
+import java.io.IOException;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.LongStream;
+
+import org.junit.Test;
+
+import org.apache.cassandra.bridge.CassandraBridgeImplementation;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.spark.data.FileType;
+import org.apache.cassandra.spark.data.SSTable;
+import org.apache.cassandra.spark.data.partitioner.Partitioner;
+import org.apache.cassandra.spark.utils.TemporaryDirectory;
+import org.apache.cassandra.spark.utils.test.TestSSTable;
+import org.apache.cassandra.spark.utils.test.TestSchema;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.quicktheories.QuickTheory.qt;
+import static org.quicktheories.generators.SourceDSL.arbitrary;
+
+public class SummaryDbTests
+{
+    private static final CassandraBridgeImplementation BRIDGE = new CassandraBridgeImplementation();
+
+    private static final class ArrayTokenList implements SummaryDbUtils.TokenList
+    {
+        private final BigInteger[] tokens;
+
+        ArrayTokenList(Long... tokens)
+        {
+            this(Arrays.stream(tokens)
+                       .map(BigInteger::valueOf)
+                       .toArray(BigInteger[]::new));
+        }
+
+        ArrayTokenList(BigInteger... tokens)
+        {
+            this.tokens = tokens;
+        }
+
+        public int size()
+        {
+            return tokens.length;
+        }
+
+        public BigInteger tokenAt(int index)
+        {
+            return tokens[index];
+        }
+    }
+
+    @Test
+    @SuppressWarnings("static-access")
+    public void testSearchSummary()
+    {
+        qt().forAll(arbitrary().enumValues(Partitioner.class))
+            .checkAssert(partitioner -> {
+                try (TemporaryDirectory directory = new TemporaryDirectory())
+                {
+                    TestSchema schema = TestSchema.basicBuilder(BRIDGE).withCompression(false).build();
+                    IPartitioner iPartitioner = BRIDGE.getPartitioner(partitioner);
+                    int numRows = 1000;
+
+                    // Write an SSTable and record token
+                    List<BigInteger> tokens = new ArrayList<>(numRows);
+                    schema.writeSSTable(directory, BRIDGE, partitioner, writer -> {
+                        for (int row = 0; row < numRows; row++)
+                        {
+                            // Cast to ByteBuffer required when compiling with Java 8
+                            ByteBuffer key = (ByteBuffer) ByteBuffer.allocate(4).putInt(row).flip();
+                            BigInteger token = ReaderUtils.tokenToBigInteger(iPartitioner.decorateKey(key).getToken());
+                            tokens.add(token);
+                            writer.write(row, 0, row);
+                        }
+                    });
+                    assertEquals(1, TestSSTable.countIn(directory.path()));
+                    Collections.sort(tokens);
+
+                    TableMetadata metadata = Schema.instance.getTableMetadata(schema.keyspace, schema.table);
+                    assertNotNull("Could not find table metadata", metadata);
+
+                    Path summaryDb = TestSSTable.firstIn(directory.path(), FileType.SUMMARY);
+                    assertNotNull("Could not find summary", summaryDb);
+
+                    SSTable ssTable = TestSSTable.firstIn(directory.path());
+                    assertNotNull("Could not find SSTable", ssTable);
+
+                    // Binary search Summary.db file in token order and verify offsets are ordered
+                    SummaryDbUtils.Summary summary = SummaryDbUtils.readSummary(metadata, ssTable);
+                    long previous = -1;
+                    for (BigInteger token : tokens)
+                    {
+                        long offset = SummaryDbUtils.findIndexOffsetInSummary(summary.summary(), iPartitioner, token);
+                        if (previous < 0)
+                        {
+                            assertEquals(offset, 0);
+                        }
+                        else
+                        {
+                            assertTrue(previous <= offset);
+                        }
+                        previous = offset;
+                    }
+                }
+                catch (IOException exception)
+                {
+                    throw new RuntimeException(exception);
+                }
+            });
+    }
+
+    @Test
+    public void testSummaryBinarySearch()
+    {
+        SummaryDbUtils.TokenList list = new ArrayTokenList(LongStream.range(5, 10000).boxed().toArray(Long[]::new));
+        assertEquals(148, SummaryDbUtils.binarySearchSummary(list, BigInteger.valueOf(154L)));
+        assertEquals(0, SummaryDbUtils.binarySearchSummary(list, BigInteger.valueOf(-500L)));
+        assertEquals(0, SummaryDbUtils.binarySearchSummary(list, BigInteger.valueOf(4L)));
+        assertEquals(0, SummaryDbUtils.binarySearchSummary(list, BigInteger.valueOf(3L)));
+        for (int token = 5; token < 10000; token++)
+        {
+            int index = SummaryDbUtils.binarySearchSummary(list, BigInteger.valueOf(token));
+            assertEquals(Math.max(0, token - 6), index);
+        }
+        assertEquals(9994, SummaryDbUtils.binarySearchSummary(list, BigInteger.valueOf(10000L)));
+        assertEquals(9994, SummaryDbUtils.binarySearchSummary(list, BigInteger.valueOf(10001L)));
+    }
+
+    @Test
+    public void testSummaryBinarySearchSparse()
+    {
+        SummaryDbUtils.TokenList list = new ArrayTokenList(5L, 10L, 15L, 20L, 25L);
+        assertEquals(0, SummaryDbUtils.binarySearchSummary(list, BigInteger.valueOf(-500L)));
+        assertEquals(0, SummaryDbUtils.binarySearchSummary(list, BigInteger.valueOf(3L)));
+        assertEquals(0, SummaryDbUtils.binarySearchSummary(list, BigInteger.valueOf(5L)));
+        assertEquals(0, SummaryDbUtils.binarySearchSummary(list, BigInteger.valueOf(6L)));
+        assertEquals(0, SummaryDbUtils.binarySearchSummary(list, BigInteger.valueOf(10L)));
+        assertEquals(1, SummaryDbUtils.binarySearchSummary(list, BigInteger.valueOf(11L)));
+        assertEquals(1, SummaryDbUtils.binarySearchSummary(list, BigInteger.valueOf(13L)));
+        assertEquals(1, SummaryDbUtils.binarySearchSummary(list, BigInteger.valueOf(15L)));
+        assertEquals(2, SummaryDbUtils.binarySearchSummary(list, BigInteger.valueOf(16L)));
+        assertEquals(3, SummaryDbUtils.binarySearchSummary(list, BigInteger.valueOf(25L)));
+        assertEquals(4, SummaryDbUtils.binarySearchSummary(list, BigInteger.valueOf(26L)));
+        assertEquals(4, SummaryDbUtils.binarySearchSummary(list, BigInteger.valueOf(100L)));
+    }
+}
diff --git a/cassandra-three-zero/TODO.md b/cassandra-three-zero/TODO.md
new file mode 100644
index 0000000..e93d2f1
--- /dev/null
+++ b/cassandra-three-zero/TODO.md
@@ -0,0 +1,7 @@
+### Note
+
+Cassandra Bridge implementation for Cassandra-all of version 3.0 is done and mostly working.
+
+It needs some adjustments in order to work with open-source Cassandra-all library (21 compilation errors).
+
+If there turns out to be public interest in having it available here, it will be open-sourced as a separate contribution.
diff --git a/code_version.sh b/code_version.sh
new file mode 100755
index 0000000..10ed2e0
--- /dev/null
+++ b/code_version.sh
@@ -0,0 +1,43 @@
+#!/bin/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.
+
+if [[ -n "$(awk -F'[ \t]*=[ \t]*' '/^version/ { print $2; exit }' gradle.properties 2>/dev/null)" ]]; then
+    version=$(awk -F'[ \t]*=[ \t]*' '/^version/ { print $2; exit }' gradle.properties)
+    from_statement="from 'gradle.properties'"
+elif [[ -n "$(awk -F'[<>]' 'NR>1 && /^[ \t]*<[^!]/ && /<version/ { print $3; exit }' pom.xml 2>/dev/null)" ]]; then
+    version=$(awk -F'[<>]' 'NR>1 && /^[ \t]*<[^!]/ && /<version/ { print $3; exit }' pom.xml)
+    from_statement="from 'pom.xml'"
+elif [[ -n "$(awk '/defproject/ { gsub("\"|-SNAPSHOT", ""); print $3 }' project.clj 2>/dev/null)" ]]; then
+    version=$(awk '/defproject/ { gsub("\"|-SNAPSHOT", ""); print $3 }' project.clj)
+    from_statement="from 'project.clj'"
+elif [[ -f version ]]; then
+    version=$(cat version)
+    from_statement="from toplevel 'version' file"
+else
+    version=$(git rev-parse --short=12 HEAD)
+    from_statement="from git hash (fallback method)"
+fi
+
+# Remove quotes from the version if present
+version=${version//\"/}
+version=${version//\'/}
+
+echo "Parsed version $from_statement: '$version'" 1>&2
+
+echo "$version"
diff --git a/config/checkstyle/checkstyle.xml b/config/checkstyle/checkstyle.xml
new file mode 100644
index 0000000..8a3dbdb
--- /dev/null
+++ b/config/checkstyle/checkstyle.xml
@@ -0,0 +1,224 @@
+<?xml version="1.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.
+-->
+<!DOCTYPE module PUBLIC
+          "-//Checkstyle//DTD Checkstyle Configuration 1.3//EN"
+          "https://checkstyle.org/dtds/configuration_1_3.dtd">
+
+<module name="Checker">
+
+    <property name="severity" value="error" />
+
+    <property name="fileExtensions" value="java, xml, properties, gradle, sh" />
+
+    <!-- Excludes all 'module-info.java' files.             -->
+    <!-- See https://checkstyle.org/config_filefilters.html -->
+    <module name="BeforeExecutionExclusionFileFilter">
+        <property name="fileNamePattern" value="module\-info\.java$" />
+    </module>
+
+    <!-- https://checkstyle.org/config_filters.html#SuppressionFilter -->
+    <module name="SuppressionFilter">
+        <property name="file" value="${config_loc}/suppressions.xml" />
+        <property name="optional" value="false" />
+    </module>
+
+    <!-- Checks that a 'package-info.java' file exists for each package. -->
+    <!-- See https://checkstyle.org/config_javadoc.html#JavadocPackage   -->
+    <module name="JavadocPackage" />
+
+    <!-- Checks for the presence of Apache license headers.   -->
+    <!-- See https://checkstyle.org/config_header.html        -->
+    <module name="RegexpHeader">
+        <property name="fileExtensions" value="java" />
+        <property name="header" value="^/\*\n \* Licensed to the Apache Software Foundation" />
+    </module>
+    <!-- TODO: Add checks for other file types (xml, properties, gradle, sh) -->
+
+    <!-- Checks whether files end with a new line.                      -->
+    <!-- See https://checkstyle.org/config_misc.html#NewlineAtEndOfFile -->
+    <module name="NewlineAtEndOfFile" />
+
+    <!-- Checks that property files contain the same keys.       -->
+    <!-- See https://checkstyle.org/config_misc.html#Translation -->
+    <module name="Translation" />
+
+    <!-- Checks for size violations.                  -->
+    <!-- See https://checkstyle.org/config_sizes.html -->
+    <module name="FileLength">
+        <property name="fileExtensions" value="java, xml, properties, gradle, sh" />
+        <property name="max" value="2400" />
+    </module>
+    <module name="LineLength">
+        <property name="fileExtensions" value="java, xml, properties, gradle, sh" />
+        <property name="ignorePattern" value="\b(package|import|Pattern\.compile|https?:://|CHECKSTYLE IGNORE)\b"/>
+        <property name="max" value="160" />
+    </module>
+
+    <!-- Checks for whitespace.                            -->
+    <!-- See https://checkstyle.org/config_whitespace.html -->
+    <module name="FileTabCharacter" />
+
+    <!-- Miscellaneous other checks.                 -->
+    <!-- See https://checkstyle.org/config_misc.html -->
+    <module name="RegexpSingleline">
+        <property name="format" value="\s+$" />
+        <property name="minimum" value="0" />
+        <property name="maximum" value="0" />
+        <property name="message" value="Line has trailing spaces." />
+    </module>
+
+    <module name="TreeWalker">
+
+        <property name="tabWidth" value="4"/>
+
+        <!-- Enables in-code single-line suppressions with a comment.                       -->
+        <!-- See https://checkstyle.org/config_filters.html#SuppressWithNearbyCommentFilter -->
+        <module name="SuppressWithNearbyCommentFilter">
+            <property name="commentFormat" value="CHECKSTYLE\s+IGNORE:\s+\S+" />
+            <property name="influenceFormat" value="-1" />
+        </module>
+        <module name="SuppressWithNearbyCommentFilter">
+            <property name="commentFormat" value="CHECKSTYLE\s+IGNORE:\s+\S+" />
+            <property name="influenceFormat" value="+1" />
+        </module>
+
+        <!-- Checks for JavaDoc comments.                   -->
+        <!-- See https://checkstyle.org/config_javadoc.html -->
+        <module name="InvalidJavadocPosition" />
+        <module name="JavadocMethod" />
+        <module name="JavadocType" />
+        <module name="JavadocVariable" />
+        <module name="JavadocStyle">
+            <property name="checkFirstSentence" value="false" />
+        </module>
+        <module name="MissingJavadocMethod" />
+
+        <!-- Checks for naming conventions.                -->
+        <!-- See https://checkstyle.org/config_naming.html -->
+        <module name="ConstantName" />
+        <module name="LocalFinalVariableName" />
+        <module name="LocalVariableName" />
+        <module name="MemberName" />
+        <module name="MethodName" />
+        <module name="PackageName" />
+        <module name="ParameterName" />
+        <module name="StaticVariableName" />
+        <module name="TypeName" />
+
+        <!-- Checks for imports.                            -->
+        <!-- See https://checkstyle.org/config_imports.html -->
+        <module name="AvoidStarImport" />
+        <module name="IllegalImport" />
+        <module name="RedundantImport" />
+        <module name="UnusedImports" />
+
+        <!-- Checks for size violations.                  -->
+        <!-- See https://checkstyle.org/config_sizes.html -->
+        <module name="MethodLength">
+            <property name="max" value="120" />
+        </module>
+        <module name="ParameterNumber">
+            <property name="max" value="8" />
+            <property name="ignoreOverriddenMethods" value="true" />
+        </module>
+
+        <!-- Checks for whitespace.                            -->
+        <!-- See https://checkstyle.org/config_whitespace.html -->
+        <module name="EmptyForIteratorPad" />
+        <module name="GenericWhitespace" />
+        <module name="MethodParamPad" />
+        <module name="NoWhitespaceAfter" />
+        <module name="NoWhitespaceBefore" />
+        <module name="OperatorWrap" />
+        <module name="ParenPad" />
+        <module name="TypecastParenPad" />
+        <module name="WhitespaceAfter" />
+        <module name="WhitespaceAround" />
+
+        <!-- Modifier checks.                                -->
+        <!-- See https://checkstyle.org/config_modifier.html -->
+        <module name="ModifierOrder" />
+        <module name="RedundantModifier" />
+
+        <!-- Checks for blocks. You know, those {}'s.      -->
+        <!-- See https://checkstyle.org/config_blocks.html -->
+        <module name="AvoidNestedBlocks" />
+        <module name="EmptyBlock" />
+        <module name="LeftCurly">
+            <property name="option" value="eol" />
+            <property name="tokens" value="LAMBDA" />
+        </module>
+        <module name="LeftCurly">
+            <property name="option" value="nl" />
+            <property name="ignoreEnums" value="false" />
+            <property name="tokens" value="ANNOTATION_DEF, CLASS_DEF, CTOR_DEF, ENUM_CONSTANT_DEF, ENUM_DEF,
+                                           INTERFACE_DEF, LITERAL_CASE, LITERAL_CATCH, LITERAL_DEFAULT,
+                                           LITERAL_DO, LITERAL_ELSE, LITERAL_FINALLY, LITERAL_FOR, LITERAL_IF,
+                                           LITERAL_SWITCH, LITERAL_SYNCHRONIZED, LITERAL_TRY, LITERAL_WHILE,
+                                           METHOD_DEF, OBJBLOCK, STATIC_INIT, RECORD_DEF, COMPACT_CTOR_DEF" />
+        </module>
+        <module name="NeedBraces" />
+        <module name="RightCurly">
+            <property name="option" value="alone" />
+        </module>
+
+        <!-- Checks for common coding problems.            -->
+        <!-- See https://checkstyle.org/config_coding.html -->
+        <module name="EmptyStatement" />
+        <module name="EqualsHashCode" />
+        <module name="HiddenField">
+            <property name="ignoreConstructorParameter" value="true" />
+            <property name="ignoreSetter" value="true" />
+        </module>
+        <module name="IllegalInstantiation" />
+        <module name="InnerAssignment" />
+        <module name="MagicNumber">
+            <property name="ignoreNumbers" value="-2147483648, -32768, -128, -1, 0, 1, 2, 3, 4, 8, 16, 32,
+                                                  42, 64, 127, 128, 255, 256, 512, 1000, 1024, 2048, 4096, 8192,
+                                                  16384, 32767, 32768, 65535, 65536, 1000000, 1048576, 1000000000,
+                                                  1073741824, 2147483647, 2147483648, 4294967295" />
+        </module>
+        <module name="MissingSwitchDefault" />
+        <module name="MultipleVariableDeclarations" />
+        <module name="SimplifyBooleanExpression" />
+        <module name="SimplifyBooleanReturn" />
+
+        <!-- Checks for class design.                      -->
+        <!-- See https://checkstyle.org/config_design.html -->
+        <module name="DesignForExtension" />
+        <module name="FinalClass" />
+        <module name="HideUtilityClassConstructor" />
+        <module name="InterfaceIsType" />
+        <module name="VisibilityModifier">
+            <property name="packageAllowed" value="true" />
+            <property name="protectedAllowed" value="true" />
+            <property name="allowPublicFinalFields" value="true" />
+            <property name="allowPublicImmutableFields" value="true" />
+        </module>
+
+        <!-- Miscellaneous other checks.                 -->
+        <!-- See https://checkstyle.org/config_misc.html -->
+        <module name="ArrayTypeStyle" />
+        <module name="TodoComment" />
+        <module name="UpperEll" />
+
+    </module>
+
+</module>
diff --git a/config/checkstyle/suppressions.xml b/config/checkstyle/suppressions.xml
new file mode 100644
index 0000000..d7584d1
--- /dev/null
+++ b/config/checkstyle/suppressions.xml
@@ -0,0 +1,40 @@
+<?xml version="1.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.
+-->
+<!DOCTYPE suppressions PUBLIC
+          "-//Checkstyle//DTD SuppressionFilter Configuration 1.1//EN"
+          "https://checkstyle.org/dtds/suppressions_1_1.dtd">
+
+<suppressions>
+
+    <!-- TODO: All the remaining code style issues should be addressed,
+               any in-place suppressions should be as narrow as possible. -->
+
+    <suppress checks="JavadocPackage" files=".*" />
+    <suppress checks="JavadocMethod" files=".*" />
+    <suppress checks="JavadocVariable" files=".*" />
+
+    <suppress checks="HiddenField" files=".*" />
+    <suppress checks="MagicNumber" files=".*" />
+
+    <suppress checks="DesignForExtension" files=".*" />
+
+    <suppress checks="TodoComment" files=".*" />
+
+</suppressions>
diff --git a/githooks/pre-push b/githooks/pre-push
new file mode 100755
index 0000000..2b9610b
--- /dev/null
+++ b/githooks/pre-push
@@ -0,0 +1,28 @@
+#!/usr/bin/env bash
+set -e
+
+PROJECT_ROOT="$(cd "$(dirname "${BASH_SOURCE[0]}")/.." && pwd)"
+
+checkstyle() {
+  SCALA_VERSION=$1
+  SPARK_VERSION=$2
+
+  export SCALA_VERSION=$SCALA_VERSION
+  export SPARK_VERSION=$SPARK_VERSION
+
+  $PROJECT_ROOT/gradlew checkstyleMain checkstyleTest
+}
+
+echo "Running pre-push hook..."
+
+# scala 2.11 && spark 2
+checkstyle 2.11 2
+
+# scala 2.12 && spark 2
+checkstyle 2.12 2
+
+# scala 2.12 && spark 3
+checkstyle 2.12 3
+
+# scala 2.12 && spark 3
+checkstyle 2.13 3
diff --git a/gradle.properties b/gradle.properties
new file mode 100644
index 0000000..6fb6ad4
--- /dev/null
+++ b/gradle.properties
@@ -0,0 +1,32 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+group=org.apache.cassandra.spark
+version=1.0.0
+snapshot=true
+description=Apache Cassandra Spark Analytics
+
+analyticsJDKLevel=11
+intellijVersion=9.0.4
+junitVersion=4.13
+quickTheoriesVersion=0.26
+mockitoVersion=3.12.4
+jnaVersion=5.9.0
+scala=2.12
+spark=3
+
+org.gradle.jvmargs=-Xmx4096m
diff --git a/gradle/wrapper/gradle-wrapper.jar b/gradle/wrapper/gradle-wrapper.jar
new file mode 100644
index 0000000..cc4fdc2
--- /dev/null
+++ b/gradle/wrapper/gradle-wrapper.jar
Binary files differ
diff --git a/gradle/wrapper/gradle-wrapper.properties b/gradle/wrapper/gradle-wrapper.properties
new file mode 100644
index 0000000..c582b66
--- /dev/null
+++ b/gradle/wrapper/gradle-wrapper.properties
@@ -0,0 +1,22 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+distributionBase=GRADLE_USER_HOME
+distributionPath=wrapper/dists
+distributionUrl=https\://services.gradle.org/distributions/gradle-7.3.3-bin.zip
+zipStoreBase=GRADLE_USER_HOME
+zipStorePath=wrapper/dists
diff --git a/gradlew b/gradlew
new file mode 100755
index 0000000..eb0e1a4
--- /dev/null
+++ b/gradlew
@@ -0,0 +1,167 @@
+#!/usr/bin/env sh
+
+##############################################################################
+##
+##  Gradle start up script for UN*X
+##
+##############################################################################
+
+# Attempt to set APP_HOME
+# Resolve links: $0 may be a link
+PRG="$0"
+# Need this for relative symlinks.
+while [ -h "$PRG" ] ; do
+    ls=`ls -ld "$PRG"`
+    link=`expr "$ls" : '.*-> \(.*\)$'`
+    if expr "$link" : '/.*' > /dev/null; then
+        PRG="$link"
+    else
+        PRG=`dirname "$PRG"`"/$link"
+    fi
+done
+SAVED="`pwd`"
+cd "`dirname \"$PRG\"`/" >/dev/null
+APP_HOME="`pwd -P`"
+cd "$SAVED" >/dev/null
+
+APP_NAME="Gradle"
+APP_BASE_NAME=`basename "$0"`
+
+# Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script.
+DEFAULT_JVM_OPTS='"-Xmx64m" "-Xms64m"'
+
+# Use the maximum available, or set MAX_FD != -1 to use that value.
+MAX_FD="maximum"
+
+warn () {
+    echo "$*"
+}
+
+die () {
+    echo
+    echo "$*"
+    echo
+    exit 1
+}
+
+# OS specific support (must be 'true' or 'false').
+cygwin=false
+msys=false
+darwin=false
+nonstop=false
+case "`uname`" in
+  CYGWIN* )
+    cygwin=true
+    ;;
+  Darwin* )
+    darwin=true
+    ;;
+  MINGW* )
+    msys=true
+    ;;
+  NONSTOP* )
+    nonstop=true
+    ;;
+esac
+
+CLASSPATH=$APP_HOME/gradle/wrapper/gradle-wrapper.jar
+
+# Determine the Java command to use to start the JVM.
+if [ -n "$JAVA_HOME" ] ; then
+    if [ -x "$JAVA_HOME/jre/sh/java" ] ; then
+        # IBM's JDK on AIX uses strange locations for the executables
+        JAVACMD="$JAVA_HOME/jre/sh/java"
+    else
+        JAVACMD="$JAVA_HOME/bin/java"
+    fi
+    if [ ! -x "$JAVACMD" ] ; then
+        die "ERROR: JAVA_HOME is set to an invalid directory: $JAVA_HOME
+
+Please set the JAVA_HOME variable in your environment to match the
+location of your Java installation."
+    fi
+else
+    JAVACMD="java"
+    which java >/dev/null 2>&1 || die "ERROR: JAVA_HOME is not set and no 'java' command could be found in your PATH.
+
+Please set the JAVA_HOME variable in your environment to match the
+location of your Java installation."
+fi
+
+# Increase the maximum file descriptors if we can.
+if [ "$cygwin" = "false" -a "$darwin" = "false" -a "$nonstop" = "false" ] ; then
+    MAX_FD_LIMIT=`ulimit -H -n`
+    if [ $? -eq 0 ] ; then
+        if [ "$MAX_FD" = "maximum" -o "$MAX_FD" = "max" ] ; then
+            MAX_FD="$MAX_FD_LIMIT"
+        fi
+        ulimit -n $MAX_FD
+        if [ $? -ne 0 ] ; then
+            warn "Could not set maximum file descriptor limit: $MAX_FD"
+        fi
+    else
+        warn "Could not query maximum file descriptor limit: $MAX_FD_LIMIT"
+    fi
+fi
+
+# For Darwin, add options to specify how the application appears in the dock
+if $darwin; then
+    GRADLE_OPTS="$GRADLE_OPTS \"-Xdock:name=$APP_NAME\" \"-Xdock:icon=$APP_HOME/media/gradle.icns\""
+fi
+
+# For Cygwin or MSYS, switch paths to Windows format before running java
+if [ "$cygwin" = "true" -o "$msys" = "true" ] ; then
+    APP_HOME=`cygpath --path --mixed "$APP_HOME"`
+    CLASSPATH=`cygpath --path --mixed "$CLASSPATH"`
+    JAVACMD=`cygpath --unix "$JAVACMD"`
+
+    # We build the pattern for arguments to be converted via cygpath
+    ROOTDIRSRAW=`find -L / -maxdepth 1 -mindepth 1 -type d 2>/dev/null`
+    SEP=""
+    for dir in $ROOTDIRSRAW ; do
+        ROOTDIRS="$ROOTDIRS$SEP$dir"
+        SEP="|"
+    done
+    OURCYGPATTERN="(^($ROOTDIRS))"
+    # Add a user-defined pattern to the cygpath arguments
+    if [ "$GRADLE_CYGPATTERN" != "" ] ; then
+        OURCYGPATTERN="$OURCYGPATTERN|($GRADLE_CYGPATTERN)"
+    fi
+    # Now convert the arguments - kludge to limit ourselves to /bin/sh
+    i=0
+    for arg in "$@" ; do
+        CHECK=`echo "$arg"|egrep -c "$OURCYGPATTERN" -`
+        CHECK2=`echo "$arg"|egrep -c "^-"`                                 ### Determine if an option
+
+        if [ $CHECK -ne 0 ] && [ $CHECK2 -eq 0 ] ; then                    ### Added a condition
+            eval `echo args$i`=`cygpath --path --ignore --mixed "$arg"`
+        else
+            eval `echo args$i`="\"$arg\""
+        fi
+        i=`expr $i + 1`
+    done
+    case $i in
+        0) set -- ;;
+        1) set -- "$args0" ;;
+        2) set -- "$args0" "$args1" ;;
+        3) set -- "$args0" "$args1" "$args2" ;;
+        4) set -- "$args0" "$args1" "$args2" "$args3" ;;
+        5) set -- "$args0" "$args1" "$args2" "$args3" "$args4" ;;
+        6) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" ;;
+        7) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" "$args6" ;;
+        8) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" "$args6" "$args7" ;;
+        9) set -- "$args0" "$args1" "$args2" "$args3" "$args4" "$args5" "$args6" "$args7" "$args8" ;;
+    esac
+fi
+
+# Escape application args
+save () {
+    for i do printf %s\\n "$i" | sed "s/'/'\\\\''/g;1s/^/'/;\$s/\$/' \\\\/" ; done
+    echo " "
+}
+APP_ARGS=$(save "$@")
+
+# Collect all arguments for the java command, following the shell quoting and substitution rules
+eval set -- $DEFAULT_JVM_OPTS $JAVA_OPTS $GRADLE_OPTS "\"-Dorg.gradle.appname=$APP_BASE_NAME\"" -classpath "\"$CLASSPATH\"" org.gradle.wrapper.GradleWrapperMain "$APP_ARGS"
+
+exec "$JAVACMD" "$@"
diff --git a/ide/idea/Project_Default.xml b/ide/idea/Project_Default.xml
new file mode 100644
index 0000000..a109884
--- /dev/null
+++ b/ide/idea/Project_Default.xml
@@ -0,0 +1,151 @@
+<!--
+    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.
+-->
+<component name="InspectionProjectProfileManager">
+  <profile version="1.0">
+    <option name="myName" value="Project Default" />
+    <option name="myLocal" value="true" />
+    <inspection_tool class="ArrayEquality" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="AssertEqualsCalledOnArray" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="AssertEqualsMayBeAssertSame" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="AutoCloseableResource" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="AwaitNotInLoop" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="AwaitWithoutCorrespondingSignal" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="BeforeClassOrAfterClassIsPublicStaticVoidNoArg" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="BeforeOrAfterIsPublicVoidNoArg" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="BigDecimalLegacyMethod" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="BusyWait" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="CStyleArrayDeclaration" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="ChannelResource" enabled="true" level="WARNING" enabled_by_default="true">
+      <option name="insideTryAllowed" value="false" />
+    </inspection_tool>
+    <inspection_tool class="CollectionContainsUrl" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="CollectionsFieldAccessReplaceableByMethodCall" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="ConfusingOctalEscape" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="Convert2streamapi" enabled="false" level="WEAK WARNING" enabled_by_default="false" />
+    <inspection_tool class="DanglingJavadoc" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="DivideByZero" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="DoubleCheckedLocking" enabled="true" level="WARNING" enabled_by_default="true">
+      <option name="ignoreOnVolatileVariables" value="false" />
+    </inspection_tool>
+    <inspection_tool class="DynamicRegexReplaceableByCompiledPattern" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="EmptySynchronizedStatement" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="EnumerationCanBeIteration" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="EqualsCalledOnEnumConstant" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="EqualsHashCodeCalledOnUrl" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="FieldCanBeLocal" enabled="true" level="WARNING" enabled_by_default="true">
+      <option name="EXCLUDE_ANNOS">
+        <value>
+          <list size="2">
+            <item index="0" class="java.lang.String" itemvalue="io.airlift.airline.Option" />
+            <item index="1" class="java.lang.String" itemvalue="org.openjdk.jmh.annotations.Param" />
+          </list>
+        </value>
+      </option>
+      <option name="IGNORE_FIELDS_USED_IN_MULTIPLE_METHODS" value="true" />
+    </inspection_tool>
+    <inspection_tool class="FieldMayBeFinal" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="FieldMayBeStatic" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="ForCanBeForeach" enabled="false" level="WARNING" enabled_by_default="false">
+      <option name="REPORT_INDEXED_LOOP" value="true" />
+      <option name="ignoreUntypedCollections" value="false" />
+    </inspection_tool>
+    <inspection_tool class="IOResource" enabled="true" level="WARNING" enabled_by_default="true">
+      <option name="ignoredTypesString" value="java.io.ByteArrayOutputStream,java.io.ByteArrayInputStream,java.io.StringBufferInputStream,java.io.CharArrayWriter,java.io.CharArrayReader,java.io.StringWriter,java.io.StringReader" />
+      <option name="insideTryAllowed" value="false" />
+    </inspection_tool>
+    <inspection_tool class="IfStatementWithIdenticalBranches" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="InnerClassMayBeStatic" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="JavaLangImport" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="KeySetIterationMayUseEntrySet" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="LengthOneStringInIndexOf" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="LengthOneStringsInConcatenation" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="LoggerInitializedWithForeignClass" enabled="false" level="WARNING" enabled_by_default="false">
+      <option name="loggerClassName" value="org.apache.log4j.Logger,org.slf4j.LoggerFactory,org.apache.commons.logging.LogFactory,java.util.logging.Logger" />
+      <option name="loggerFactoryMethodName" value="getLogger,getLogger,getLog,getLogger" />
+    </inspection_tool>
+    <inspection_tool class="LoggingConditionDisagreesWithLogStatement" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="MapReplaceableByEnumMap" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="NonStaticFinalLogger" enabled="true" level="WARNING" enabled_by_default="true">
+      <option name="loggerClassName" value="" />
+    </inspection_tool>
+    <inspection_tool class="NullThrown" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="NullableProblems" enabled="false" level="WARNING" enabled_by_default="false">
+      <option name="REPORT_NULLABLE_METHOD_OVERRIDES_NOTNULL" value="true" />
+      <option name="REPORT_NOT_ANNOTATED_METHOD_OVERRIDES_NOTNULL" value="true" />
+      <option name="REPORT_NOTNULL_PARAMETER_OVERRIDES_NULLABLE" value="true" />
+      <option name="REPORT_NOT_ANNOTATED_PARAMETER_OVERRIDES_NOTNULL" value="true" />
+      <option name="REPORT_NOT_ANNOTATED_GETTER" value="true" />
+      <option name="REPORT_NOT_ANNOTATED_SETTER_PARAMETER" value="true" />
+      <option name="REPORT_ANNOTATION_NOT_PROPAGATED_TO_OVERRIDERS" value="true" />
+      <option name="REPORT_NULLS_PASSED_TO_NON_ANNOTATED_METHOD" value="true" />
+    </inspection_tool>
+    <inspection_tool class="ObjectToString" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="ObsoleteCollection" enabled="true" level="WARNING" enabled_by_default="true">
+      <option name="ignoreRequiredObsoleteCollectionTypes" value="false" />
+    </inspection_tool>
+    <inspection_tool class="PointlessIndexOfComparison" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="PyArgumentListInspection" enabled="false" level="WARNING" enabled_by_default="false" />
+    <inspection_tool class="PyPep8Inspection" enabled="false" level="WEAK WARNING" enabled_by_default="false">
+      <option name="ignoredErrors">
+        <list>
+          <option value="E501" />
+        </list>
+      </option>
+    </inspection_tool>
+    <inspection_tool class="PyPep8NamingInspection" enabled="false" level="WEAK WARNING" enabled_by_default="false" />
+    <inspection_tool class="PyRedundantParenthesesInspection" enabled="false" level="WEAK WARNING" enabled_by_default="false">
+      <option name="myIgnorePercOperator" value="false" />
+      <option name="myIgnoreTupleInReturn" value="false" />
+    </inspection_tool>
+    <inspection_tool class="PySetFunctionToLiteralInspection" enabled="false" level="WARNING" enabled_by_default="false" />
+    <inspection_tool class="PyShadowingBuiltinsInspection" enabled="false" level="WEAK WARNING" enabled_by_default="false" />
+    <inspection_tool class="PyTrailingSemicolonInspection" enabled="false" level="WARNING" enabled_by_default="false" />
+    <inspection_tool class="RedundantThrowsDeclaration" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="SafeLock" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="SetReplaceableByEnumSet" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="SocketResource" enabled="true" level="WARNING" enabled_by_default="true">
+      <option name="insideTryAllowed" value="false" />
+    </inspection_tool>
+    <inspection_tool class="SpellCheckingInspection" enabled="false" level="TYPO" enabled_by_default="false">
+      <option name="processCode" value="true" />
+      <option name="processLiterals" value="true" />
+      <option name="processComments" value="true" />
+    </inspection_tool>
+    <inspection_tool class="SqlDialectInspection" enabled="false" level="WARNING" enabled_by_default="false" />
+    <inspection_tool class="SqlNoDataSourceInspection" enabled="false" level="WARNING" enabled_by_default="false" />
+    <inspection_tool class="StaticPseudoFunctionalStyleMethod" enabled="false" level="WARNING" enabled_by_default="false" />
+    <inspection_tool class="StringBufferToStringInConcatenation" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="StringConcatenationInLoops" enabled="true" level="WARNING" enabled_by_default="true">
+      <option name="m_ignoreUnlessAssigned" value="true" />
+    </inspection_tool>
+    <inspection_tool class="StringEqualsEmptyString" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="SystemGC" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="SystemOutErr" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="TailRecursion" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="ThreadDumpStack" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="ThrowablePrintStackTrace" enabled="true" level="WARNING" enabled_by_default="true">
+      <scope name="Tests" level="WARNING" enabled="false" />
+    </inspection_tool>
+    <inspection_tool class="TrivialStringConcatenation" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="UnnecessaryCallToStringValueOf" enabled="true" level="WARNING" enabled_by_default="true" />
+    <inspection_tool class="UnnecessaryInterfaceModifier" enabled="false" level="WARNING" enabled_by_default="false" />
+    <inspection_tool class="WhileCanBeForeach" enabled="false" level="WARNING" enabled_by_default="false" />
+    <inspection_tool class="ZeroLengthArrayInitialization" enabled="true" level="WARNING" enabled_by_default="true" />
+  </profile>
+</component>
diff --git a/ide/idea/codeStyleSettings.xml b/ide/idea/codeStyleSettings.xml
new file mode 100644
index 0000000..25d3c0e
--- /dev/null
+++ b/ide/idea/codeStyleSettings.xml
@@ -0,0 +1,349 @@
+<?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.
+-->
+<code_scheme name="Apache Cassandra" version="173">
+  <option name="RIGHT_MARGIN" value="160" />
+  <option name="BRACE_STYLE" value="2" />
+  <option name="CLASS_BRACE_STYLE" value="2" />
+  <option name="METHOD_BRACE_STYLE" value="2" />
+  <option name="ELSE_ON_NEW_LINE" value="true" />
+  <option name="CATCH_ON_NEW_LINE" value="true" />
+  <option name="FINALLY_ON_NEW_LINE" value="true" />
+  <option name="ALIGN_MULTILINE_PARAMETERS_IN_CALLS" value="true" />
+  <option name="ALIGN_MULTILINE_BINARY_OPERATION" value="true" />
+  <option name="ALIGN_MULTILINE_ASSIGNMENT" value="true" />
+  <option name="ALIGN_MULTILINE_TERNARY_OPERATION" value="true" />
+  <option name="ALIGN_MULTILINE_THROWS_LIST" value="true" />
+  <option name="ALIGN_MULTILINE_EXTENDS_LIST" value="true" />
+  <option name="ALIGN_MULTILINE_PARENTHESIZED_EXPRESSION" value="true" />
+  <option name="ALIGN_MULTILINE_ARRAY_INITIALIZER_EXPRESSION" value="true" />
+  <option name="SPACE_WITHIN_ARRAY_INITIALIZER_BRACES" value="true" />
+  <option name="DOWHILE_BRACE_FORCE" value="3" />
+  <option name="SOFT_MARGINS" value="120" />
+  <JavaCodeStyleSettings>
+    <option name="CLASS_COUNT_TO_USE_IMPORT_ON_DEMAND" value="999" />
+    <option name="NAMES_COUNT_TO_USE_IMPORT_ON_DEMAND" value="999" />
+    <option name="IMPORT_LAYOUT_TABLE">
+      <value>
+        <package name="java" withSubpackages="true" static="false" />
+        <package name="javax" withSubpackages="true" static="false" />
+        <emptyLine />
+        <package name="com.google.common" withSubpackages="true" static="false" />
+        <package name="org.apache.log4j" withSubpackages="true" static="false" />
+        <package name="org.apache.commons" withSubpackages="true" static="false" />
+        <package name="org.cliffc.high_scale_lib" withSubpackages="true" static="false" />
+        <package name="org.junit" withSubpackages="true" static="false" />
+        <package name="org.slf4j" withSubpackages="true" static="false" />
+        <emptyLine />
+        <package name="" withSubpackages="true" static="false" />
+        <emptyLine />
+        <package name="" withSubpackages="true" static="true" />
+      </value>
+    </option>
+  </JavaCodeStyleSettings>
+  <codeStyleSettings language="JAVA">
+    <option name="RIGHT_MARGIN" value="160" />
+    <option name="KEEP_BLANK_LINES_BEFORE_RBRACE" value="0" />
+    <option name="BRACE_STYLE" value="2" />
+    <option name="CLASS_BRACE_STYLE" value="2" />
+    <option name="METHOD_BRACE_STYLE" value="2" />
+    <option name="ELSE_ON_NEW_LINE" value="true" />
+    <option name="CATCH_ON_NEW_LINE" value="true" />
+    <option name="FINALLY_ON_NEW_LINE" value="true" />
+    <option name="ALIGN_MULTILINE_CHAINED_METHODS" value="true" />
+    <option name="ALIGN_MULTILINE_PARAMETERS_IN_CALLS" value="true" />
+    <option name="ALIGN_MULTILINE_BINARY_OPERATION" value="true" />
+    <option name="ALIGN_MULTILINE_ASSIGNMENT" value="true" />
+    <option name="ALIGN_MULTILINE_TERNARY_OPERATION" value="true" />
+    <option name="ALIGN_MULTILINE_THROWS_LIST" value="true" />
+    <option name="ALIGN_THROWS_KEYWORD" value="true" />
+    <option name="ALIGN_MULTILINE_EXTENDS_LIST" value="true" />
+    <option name="ALIGN_MULTILINE_PARENTHESIZED_EXPRESSION" value="true" />
+    <option name="ALIGN_MULTILINE_ARRAY_INITIALIZER_EXPRESSION" value="true" />
+    <option name="SPACE_WITHIN_ARRAY_INITIALIZER_BRACES" value="true" />
+    <option name="SPACE_BEFORE_ARRAY_INITIALIZER_LBRACE" value="true" />
+    <option name="CALL_PARAMETERS_WRAP" value="1" />
+    <option name="METHOD_PARAMETERS_WRAP" value="1" />
+    <option name="RESOURCE_LIST_WRAP" value="1" />
+    <option name="EXTENDS_LIST_WRAP" value="1" />
+    <option name="THROWS_LIST_WRAP" value="1" />
+    <option name="EXTENDS_KEYWORD_WRAP" value="1" />
+    <option name="THROWS_KEYWORD_WRAP" value="1" />
+    <option name="METHOD_CALL_CHAIN_WRAP" value="1" />
+    <option name="BINARY_OPERATION_WRAP" value="1" />
+    <option name="TERNARY_OPERATION_WRAP" value="1" />
+    <option name="FOR_STATEMENT_WRAP" value="1" />
+    <option name="ARRAY_INITIALIZER_WRAP" value="1" />
+    <option name="ASSIGNMENT_WRAP" value="1" />
+    <option name="ASSERT_STATEMENT_WRAP" value="1" />
+    <option name="IF_BRACE_FORCE" value="3" />
+    <option name="DOWHILE_BRACE_FORCE" value="3" />
+    <option name="WHILE_BRACE_FORCE" value="3" />
+    <option name="FOR_BRACE_FORCE" value="3" />
+    <option name="PARAMETER_ANNOTATION_WRAP" value="1" />
+    <option name="VARIABLE_ANNOTATION_WRAP" value="1" />
+    <option name="ENUM_CONSTANTS_WRAP" value="1" />
+    <indentOptions>
+      <option name="CONTINUATION_INDENT_SIZE" value="0" />
+      <option name="USE_RELATIVE_INDENTS" value="true" />
+    </indentOptions>
+    <arrangement>
+      <rules>
+        <section>
+          <rule>
+            <match>
+              <AND>
+                <FIELD />
+                <FINAL />
+                <PUBLIC />
+                <STATIC />
+              </AND>
+            </match>
+          </rule>
+        </section>
+        <section>
+          <rule>
+            <match>
+              <AND>
+                <FIELD />
+                <FINAL />
+                <PROTECTED />
+                <STATIC />
+              </AND>
+            </match>
+          </rule>
+        </section>
+        <section>
+          <rule>
+            <match>
+              <AND>
+                <FIELD />
+                <FINAL />
+                <PACKAGE_PRIVATE />
+                <STATIC />
+              </AND>
+            </match>
+          </rule>
+        </section>
+        <section>
+          <rule>
+            <match>
+              <AND>
+                <FIELD />
+                <FINAL />
+                <PRIVATE />
+                <STATIC />
+              </AND>
+            </match>
+          </rule>
+        </section>
+        <section>
+          <rule>
+            <match>
+              <AND>
+                <FIELD />
+                <PUBLIC />
+                <STATIC />
+              </AND>
+            </match>
+          </rule>
+        </section>
+        <section>
+          <rule>
+            <match>
+              <AND>
+                <FIELD />
+                <PROTECTED />
+                <STATIC />
+              </AND>
+            </match>
+          </rule>
+        </section>
+        <section>
+          <rule>
+            <match>
+              <AND>
+                <FIELD />
+                <PACKAGE_PRIVATE />
+                <STATIC />
+              </AND>
+            </match>
+          </rule>
+        </section>
+        <section>
+          <rule>
+            <match>
+              <AND>
+                <FIELD />
+                <PRIVATE />
+                <STATIC />
+              </AND>
+            </match>
+          </rule>
+        </section>
+        <section>
+          <rule>
+            <match>
+              <AND>
+                <FIELD />
+                <FINAL />
+                <PUBLIC />
+              </AND>
+            </match>
+          </rule>
+        </section>
+        <section>
+          <rule>
+            <match>
+              <AND>
+                <FIELD />
+                <FINAL />
+                <PROTECTED />
+              </AND>
+            </match>
+          </rule>
+        </section>
+        <section>
+          <rule>
+            <match>
+              <AND>
+                <FIELD />
+                <FINAL />
+                <PACKAGE_PRIVATE />
+              </AND>
+            </match>
+          </rule>
+        </section>
+        <section>
+          <rule>
+            <match>
+              <AND>
+                <FIELD />
+                <FINAL />
+                <PRIVATE />
+              </AND>
+            </match>
+          </rule>
+        </section>
+        <section>
+          <rule>
+            <match>
+              <AND>
+                <FIELD />
+                <PUBLIC />
+              </AND>
+            </match>
+          </rule>
+        </section>
+        <section>
+          <rule>
+            <match>
+              <AND>
+                <FIELD />
+                <PROTECTED />
+              </AND>
+            </match>
+          </rule>
+        </section>
+        <section>
+          <rule>
+            <match>
+              <AND>
+                <FIELD />
+                <PACKAGE_PRIVATE />
+              </AND>
+            </match>
+          </rule>
+        </section>
+        <section>
+          <rule>
+            <match>
+              <AND>
+                <FIELD />
+                <PRIVATE />
+              </AND>
+            </match>
+          </rule>
+        </section>
+        <section>
+          <rule>
+            <match>
+              <FIELD />
+            </match>
+          </rule>
+        </section>
+        <section>
+          <rule>
+            <match>
+              <CONSTRUCTOR />
+            </match>
+          </rule>
+        </section>
+        <section>
+          <rule>
+            <match>
+              <AND>
+                <METHOD />
+                <STATIC />
+              </AND>
+            </match>
+          </rule>
+        </section>
+        <section>
+          <rule>
+            <match>
+              <METHOD />
+            </match>
+          </rule>
+        </section>
+        <section>
+          <rule>
+            <match>
+              <ENUM />
+            </match>
+          </rule>
+        </section>
+        <section>
+          <rule>
+            <match>
+              <INTERFACE />
+            </match>
+          </rule>
+        </section>
+        <section>
+          <rule>
+            <match>
+              <AND>
+                <CLASS />
+                <STATIC />
+              </AND>
+            </match>
+          </rule>
+        </section>
+        <section>
+          <rule>
+            <match>
+              <CLASS />
+            </match>
+          </rule>
+        </section>
+      </rules>
+    </arrangement>
+  </codeStyleSettings>
+</code_scheme>
diff --git a/profiles/scala-2.11-spark-2-jdk-1.8.gradle b/profiles/scala-2.11-spark-2-jdk-1.8.gradle
new file mode 100644
index 0000000..2e469fa
--- /dev/null
+++ b/profiles/scala-2.11-spark-2-jdk-1.8.gradle
@@ -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.
+ */
+
+ext {
+    jacksonVersion="2.6.7.5"
+    scalaMajorVersion="2.11"
+    scalaVersion="2.11.12"
+    sparkGroupId="org.apache.spark"
+    sparkMajorVersion="2"
+    sparkVersion="2.4.8"
+    sidecarVersion="1.0.0-jdk8-local"
+}
diff --git a/profiles/scala-2.12-spark-2-jdk-1.8.gradle b/profiles/scala-2.12-spark-2-jdk-1.8.gradle
new file mode 100644
index 0000000..832a725
--- /dev/null
+++ b/profiles/scala-2.12-spark-2-jdk-1.8.gradle
@@ -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.
+ */
+
+ext {
+    jacksonVersion="2.6.7.3"
+    scalaMajorVersion="2.12"
+    scalaVersion="2.12.10"
+    sparkGroupId="org.apache.spark"
+    sparkMajorVersion="2"
+    sparkVersion="2.4.8"
+    sidecarVersion="1.0.0-jdk8-local"
+}
diff --git a/profiles/scala-2.12-spark-3-jdk-11.gradle b/profiles/scala-2.12-spark-3-jdk-11.gradle
new file mode 100644
index 0000000..ca60c7a
--- /dev/null
+++ b/profiles/scala-2.12-spark-3-jdk-11.gradle
@@ -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.
+ */
+
+ext {
+    jacksonVersion="2.14.2"
+    scalaMajorVersion="2.12"
+    scalaVersion="2.12.10"
+    sparkGroupId="org.apache.spark"
+    sparkMajorVersion="3"
+    sparkVersion="3.2.2"
+    sidecarVersion="1.0.0-local"
+}
diff --git a/profiles/scala-2.13-spark-3-jdk-11.gradle b/profiles/scala-2.13-spark-3-jdk-11.gradle
new file mode 100644
index 0000000..a8e27f3
--- /dev/null
+++ b/profiles/scala-2.13-spark-3-jdk-11.gradle
@@ -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.
+ */
+
+ext {
+    jacksonVersion="2.12.7"
+    scalaMajorVersion="2.13"
+    scalaVersion="2.13.10"
+    sparkGroupId="org.apache.spark"
+    sparkMajorVersion="3"
+    sparkVersion="3.2.2"
+    sidecarVersion="1.0.0-local"
+}
diff --git a/settings.gradle b/settings.gradle
new file mode 100644
index 0000000..704e124
--- /dev/null
+++ b/settings.gradle
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+
+rootProject.name = 'cassandra-analytics-core'
+
+include 'cassandra-bridge'
+include 'cassandra-four-zero'
+include 'cassandra-analytics-core'
+include 'cassandra-analytics-core-example'