[HUDI-1089] Refactor hudi-client to support multi-engine (#1827)

- This change breaks `hudi-client` into `hudi-client-common` and `hudi-spark-client` modules 
- Simple usages of Spark using jsc.parallelize() has been redone using EngineContext#map, EngineContext#flatMap etc
- Code changes in the PR, break classes into `BaseXYZ` parent classes with no spark dependencies living in `hudi-client-common`
- Classes on `hudi-spark-client` are named `SparkXYZ` extending the parent classes with all the Spark dependencies
- To simplify/cleanup, HoodieIndex#fetchRecordLocation has been removed and its usages in tests replaced with alternatives

Co-authored-by: Vinoth Chandar <vinoth@apache.org>
diff --git a/.travis.yml b/.travis.yml
index 67fa007..d36c0cb 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -20,10 +20,10 @@
   - openjdk8
 jobs:
   include:
-    - name: "Unit tests except hudi-client"
-      env: MODE=unit MODULES='!hudi-client' HUDI_QUIETER_LOGGING=1
-    - name: "Unit tests for hudi-client"
-      env: MODE=unit MODULES=hudi-client HUDI_QUIETER_LOGGING=1
+    - name: "Unit tests except hudi-spark-client"
+      env: MODE=unit MODULES='!hudi-client/hudi-spark-client' HUDI_QUIETER_LOGGING=1
+    - name: "Unit tests for hudi-spark-client"
+      env: MODE=unit MODULES=hudi-client/hudi-spark-client HUDI_QUIETER_LOGGING=1
     - name: "Functional tests"
       env: MODE=functional HUDI_QUIETER_LOGGING=1
     - name: "Integration tests"
diff --git a/hudi-cli/pom.xml b/hudi-cli/pom.xml
index a1b398e..dda07f9 100644
--- a/hudi-cli/pom.xml
+++ b/hudi-cli/pom.xml
@@ -148,7 +148,14 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hudi</groupId>
-      <artifactId>hudi-client</artifactId>
+      <artifactId>hudi-client-common</artifactId>
+      <version>${project.version}</version>
+      <scope>test</scope>
+      <type>test-jar</type>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hudi</groupId>
+      <artifactId>hudi-spark-client</artifactId>
       <version>${project.version}</version>
       <scope>test</scope>
       <type>test-jar</type>
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SavepointsCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SavepointsCommand.java
index 6d81552..e4d7cc6 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SavepointsCommand.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SavepointsCommand.java
@@ -23,7 +23,8 @@
 import org.apache.hudi.cli.HoodieTableHeaderFields;
 import org.apache.hudi.cli.utils.InputStreamConsumer;
 import org.apache.hudi.cli.utils.SparkUtil;
-import org.apache.hudi.client.HoodieWriteClient;
+import org.apache.hudi.client.SparkRDDWriteClient;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
 import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
 import org.apache.hudi.common.table.timeline.HoodieInstant;
@@ -162,10 +163,10 @@
     return String.format("Savepoint \"%s\" deleted.", instantTime);
   }
 
-  private static HoodieWriteClient createHoodieClient(JavaSparkContext jsc, String basePath) throws Exception {
+  private static SparkRDDWriteClient createHoodieClient(JavaSparkContext jsc, String basePath) throws Exception {
     HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
         .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()).build();
-    return new HoodieWriteClient(jsc, config, false);
+    return new SparkRDDWriteClient(new HoodieSparkEngineContext(jsc), config, false);
   }
 
 }
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java
index ce567b9..f715b16 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java
@@ -22,12 +22,13 @@
 import org.apache.hudi.DataSourceWriteOptions;
 import org.apache.hudi.cli.DedupeSparkJob;
 import org.apache.hudi.cli.utils.SparkUtil;
-import org.apache.hudi.client.HoodieWriteClient;
+import org.apache.hudi.client.SparkRDDWriteClient;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.config.TypedProperties;
-import org.apache.hudi.client.utils.ClientUtils;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
 import org.apache.hudi.common.table.HoodieTableVersion;
+import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.StringUtils;
 import org.apache.hudi.config.HoodieBootstrapConfig;
@@ -35,8 +36,8 @@
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.exception.HoodieSavepointException;
 import org.apache.hudi.index.HoodieIndex;
-import org.apache.hudi.table.upgrade.UpgradeDowngrade;
 import org.apache.hudi.table.action.compact.strategy.UnBoundedCompactionStrategy;
+import org.apache.hudi.table.upgrade.SparkUpgradeDowngrade;
 import org.apache.hudi.utilities.HDFSParquetImporter;
 import org.apache.hudi.utilities.HDFSParquetImporter.Config;
 import org.apache.hudi.utilities.HoodieCleaner;
@@ -343,7 +344,7 @@
   }
 
   private static int rollback(JavaSparkContext jsc, String instantTime, String basePath) throws Exception {
-    HoodieWriteClient client = createHoodieClient(jsc, basePath);
+    SparkRDDWriteClient client = createHoodieClient(jsc, basePath);
     if (client.rollback(instantTime)) {
       LOG.info(String.format("The commit \"%s\" rolled back.", instantTime));
       return 0;
@@ -355,7 +356,7 @@
 
   private static int createSavepoint(JavaSparkContext jsc, String commitTime, String user,
       String comments, String basePath) throws Exception {
-    HoodieWriteClient client = createHoodieClient(jsc, basePath);
+    SparkRDDWriteClient client = createHoodieClient(jsc, basePath);
     try {
       client.savepoint(commitTime, user, comments);
       LOG.info(String.format("The commit \"%s\" has been savepointed.", commitTime));
@@ -367,7 +368,7 @@
   }
 
   private static int rollbackToSavepoint(JavaSparkContext jsc, String savepointTime, String basePath) throws Exception {
-    HoodieWriteClient client = createHoodieClient(jsc, basePath);
+    SparkRDDWriteClient client = createHoodieClient(jsc, basePath);
     try {
       client.restoreToSavepoint(savepointTime);
       LOG.info(String.format("The commit \"%s\" rolled back.", savepointTime));
@@ -379,7 +380,7 @@
   }
 
   private static int deleteSavepoint(JavaSparkContext jsc, String savepointTime, String basePath) throws Exception {
-    HoodieWriteClient client = createHoodieClient(jsc, basePath);
+    SparkRDDWriteClient client = createHoodieClient(jsc, basePath);
     try {
       client.deleteSavepoint(savepointTime);
       LOG.info(String.format("Savepoint \"%s\" deleted.", savepointTime));
@@ -401,9 +402,10 @@
    */
   protected static int upgradeOrDowngradeTable(JavaSparkContext jsc, String basePath, String toVersion) {
     HoodieWriteConfig config = getWriteConfig(basePath);
-    HoodieTableMetaClient metaClient = ClientUtils.createMetaClient(jsc.hadoopConfiguration(), config, false);
+    HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), false,
+        config.getConsistencyGuardConfig(), Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion())));
     try {
-      UpgradeDowngrade.run(metaClient, HoodieTableVersion.valueOf(toVersion), config, jsc, null);
+      new SparkUpgradeDowngrade(metaClient, config, new HoodieSparkEngineContext(jsc)).run(metaClient, HoodieTableVersion.valueOf(toVersion), config, new HoodieSparkEngineContext(jsc), null);
       LOG.info(String.format("Table at \"%s\" upgraded / downgraded to version \"%s\".", basePath, toVersion));
       return 0;
     } catch (Exception e) {
@@ -412,9 +414,9 @@
     }
   }
 
-  private static HoodieWriteClient createHoodieClient(JavaSparkContext jsc, String basePath) throws Exception {
+  private static SparkRDDWriteClient createHoodieClient(JavaSparkContext jsc, String basePath) throws Exception {
     HoodieWriteConfig config = getWriteConfig(basePath);
-    return new HoodieWriteClient(jsc, config);
+    return new SparkRDDWriteClient(new HoodieSparkEngineContext(jsc), config);
   }
 
   private static HoodieWriteConfig getWriteConfig(String basePath) {
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/utils/SparkUtil.java b/hudi-cli/src/main/java/org/apache/hudi/cli/utils/SparkUtil.java
index ba9d1e5..430aab9 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/utils/SparkUtil.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/utils/SparkUtil.java
@@ -21,7 +21,7 @@
 import org.apache.hudi.cli.HoodieCliSparkConfig;
 import org.apache.hudi.cli.commands.SparkEnvCommand;
 import org.apache.hudi.cli.commands.SparkMain;
-import org.apache.hudi.client.HoodieWriteClient;
+import org.apache.hudi.client.SparkRDDWriteClient;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.StringUtils;
@@ -91,7 +91,7 @@
     sparkConf.set(HoodieCliSparkConfig.CLI_MAPRED_OUTPUT_COMPRESSION_CODEC, "org.apache.hadoop.io.compress.GzipCodec");
     sparkConf.set(HoodieCliSparkConfig.CLI_MAPRED_OUTPUT_COMPRESSION_TYPE, "BLOCK");
 
-    HoodieWriteClient.registerClasses(sparkConf);
+    SparkRDDWriteClient.registerClasses(sparkConf);
     JavaSparkContext jsc = new JavaSparkContext(sparkConf);
     jsc.hadoopConfiguration().setBoolean(HoodieCliSparkConfig.CLI_PARQUET_ENABLE_SUMMARY_METADATA, false);
     FSUtils.prepareHadoopConf(jsc.hadoopConfiguration());
diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestArchivedCommitsCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestArchivedCommitsCommand.java
index f69e3d3..5c27636 100644
--- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestArchivedCommitsCommand.java
+++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestArchivedCommitsCommand.java
@@ -30,6 +30,7 @@
 import org.apache.hudi.common.table.timeline.HoodieTimeline;
 import org.apache.hudi.config.HoodieCompactionConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.table.HoodieSparkTable;
 import org.apache.hudi.table.HoodieTimelineArchiveLog;
 
 import org.junit.jupiter.api.AfterEach;
@@ -92,8 +93,9 @@
     metaClient.getActiveTimeline().reload().getAllCommitsTimeline().filterCompletedInstants();
 
     // archive
-    HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, hadoopConf);
-    archiveLog.archiveIfRequired(jsc);
+    HoodieSparkTable table = HoodieSparkTable.create(cfg, context, metaClient);
+    HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table);
+    archiveLog.archiveIfRequired(context);
   }
 
   @AfterEach
diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCommitsCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCommitsCommand.java
index c2509b1..84b3576 100644
--- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCommitsCommand.java
+++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCommitsCommand.java
@@ -35,6 +35,7 @@
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.config.HoodieCompactionConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.table.HoodieSparkTable;
 import org.apache.hudi.table.HoodieTimelineArchiveLog;
 
 import org.apache.hadoop.fs.FileSystem;
@@ -179,8 +180,9 @@
 
     // archive
     metaClient = HoodieTableMetaClient.reload(HoodieCLI.getTableMetaClient());
-    HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, jsc.hadoopConfiguration());
-    archiveLog.archiveIfRequired(jsc);
+    HoodieSparkTable table = HoodieSparkTable.create(cfg, context, metaClient);
+    HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table);
+    archiveLog.archiveIfRequired(context);
 
     CommandResult cr = getShell().executeCommand(String.format("commits showarchived --startTs %s --endTs %s", "100", "104"));
     assertTrue(cr.isSuccess());
diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestRollbacksCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestRollbacksCommand.java
index 10197a3..8cf2be9 100644
--- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestRollbacksCommand.java
+++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestRollbacksCommand.java
@@ -24,7 +24,7 @@
 import org.apache.hudi.cli.HoodieTableHeaderFields;
 import org.apache.hudi.cli.TableHeader;
 import org.apache.hudi.cli.testutils.AbstractShellIntegrationTest;
-import org.apache.hudi.client.HoodieWriteClient;
+import org.apache.hudi.client.AbstractHoodieWriteClient;
 import org.apache.hudi.common.model.HoodieTableType;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
 import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
@@ -90,7 +90,7 @@
     HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(tablePath)
         .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build();
 
-    try (HoodieWriteClient client = getHoodieWriteClient(config)) {
+    try (AbstractHoodieWriteClient client = getHoodieWriteClient(config)) {
       // Rollback inflight commit3 and commit2
       client.rollback("102");
       client.rollback("101");
diff --git a/hudi-client/hudi-client-common/pom.xml b/hudi-client/hudi-client-common/pom.xml
new file mode 100644
index 0000000..902de58
--- /dev/null
+++ b/hudi-client/hudi-client-common/pom.xml
@@ -0,0 +1,264 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <parent>
+    <artifactId>hudi-client</artifactId>
+    <groupId>org.apache.hudi</groupId>
+    <version>0.6.1-SNAPSHOT</version>
+  </parent>
+  <modelVersion>4.0.0</modelVersion>
+
+  <artifactId>hudi-client-common</artifactId>
+  <version>${parent.version}</version>
+
+  <name>hudi-client-common</name>
+  <packaging>jar</packaging>
+
+  <dependencies>
+    <!-- Scala -->
+    <dependency>
+      <groupId>org.scala-lang</groupId>
+      <artifactId>scala-library</artifactId>
+      <version>${scala.version}</version>
+    </dependency>
+
+    <!-- Hoodie -->
+    <dependency>
+      <groupId>org.apache.hudi</groupId>
+      <artifactId>hudi-common</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hudi</groupId>
+      <artifactId>hudi-timeline-service</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+
+    <!-- Logging -->
+    <dependency>
+      <groupId>log4j</groupId>
+      <artifactId>log4j</artifactId>
+    </dependency>
+
+    <!-- Parquet -->
+    <dependency>
+      <groupId>org.apache.parquet</groupId>
+      <artifactId>parquet-avro</artifactId>
+    </dependency>
+
+    <!-- Dropwizard Metrics -->
+    <dependency>
+      <groupId>io.dropwizard.metrics</groupId>
+      <artifactId>metrics-graphite</artifactId>
+      <exclusions>
+        <exclusion>
+          <groupId>com.rabbitmq</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>io.dropwizard.metrics</groupId>
+      <artifactId>metrics-core</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>io.dropwizard.metrics</groupId>
+      <artifactId>metrics-jmx</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>io.prometheus</groupId>
+      <artifactId>simpleclient</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>io.prometheus</groupId>
+      <artifactId>simpleclient_httpserver</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>io.prometheus</groupId>
+      <artifactId>simpleclient_dropwizard</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>io.prometheus</groupId>
+      <artifactId>simpleclient_pushgateway</artifactId>
+    </dependency>
+
+    <!-- Hoodie - Tests -->
+    <dependency>
+      <groupId>org.apache.hudi</groupId>
+      <artifactId>hudi-common</artifactId>
+      <version>${project.version}</version>
+      <classifier>tests</classifier>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+
+    <!-- Hadoop -->
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-hdfs</artifactId>
+      <classifier>tests</classifier>
+      <!-- Need these exclusions to make sure JavaSparkContext can be setup. https://issues.apache.org/jira/browse/SPARK-1693 -->
+      <exclusions>
+        <exclusion>
+          <groupId>org.mortbay.jetty</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>javax.servlet.jsp</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>javax.servlet</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <classifier>tests</classifier>
+      <exclusions>
+        <exclusion>
+          <groupId>org.mortbay.jetty</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>javax.servlet.jsp</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>javax.servlet</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <!-- Test -->
+    <dependency>
+      <groupId>org.junit.jupiter</groupId>
+      <artifactId>junit-jupiter-api</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.junit.jupiter</groupId>
+      <artifactId>junit-jupiter-engine</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.junit.vintage</groupId>
+      <artifactId>junit-vintage-engine</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.junit.jupiter</groupId>
+      <artifactId>junit-jupiter-params</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-junit-jupiter</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.junit.platform</groupId>
+      <artifactId>junit-platform-runner</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.junit.platform</groupId>
+      <artifactId>junit-platform-suite-api</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.junit.platform</groupId>
+      <artifactId>junit-platform-commons</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.jacoco</groupId>
+        <artifactId>jacoco-maven-plugin</artifactId>
+      </plugin>
+      <plugin>
+        <groupId>net.alchim31.maven</groupId>
+        <artifactId>scala-maven-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>scala-compile-first</id>
+            <phase>process-resources</phase>
+            <goals>
+              <goal>add-source</goal>
+              <goal>compile</goal>
+            </goals>
+          </execution>
+          <execution>
+            <id>scala-test-compile</id>
+            <phase>process-test-resources</phase>
+            <goals>
+              <goal>testCompile</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <executions>
+          <execution>
+            <phase>compile</phase>
+            <goals>
+              <goal>compile</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+            <phase>test-compile</phase>
+          </execution>
+        </executions>
+        <configuration>
+          <skip>false</skip>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.rat</groupId>
+        <artifactId>apache-rat-plugin</artifactId>
+      </plugin>
+    </plugins>
+
+    <resources>
+      <resource>
+        <directory>src/main/resources</directory>
+      </resource>
+      <resource>
+        <directory>src/test/resources</directory>
+      </resource>
+    </resources>
+  </build>
+</project>
diff --git a/hudi-client/src/main/java/org/apache/hudi/async/AsyncCompactService.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/AsyncCompactService.java
similarity index 83%
rename from hudi-client/src/main/java/org/apache/hudi/async/AsyncCompactService.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/AsyncCompactService.java
index 391344f..d7faf54 100644
--- a/hudi-client/src/main/java/org/apache/hudi/async/AsyncCompactService.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/AsyncCompactService.java
@@ -17,14 +17,15 @@
 
 package org.apache.hudi.async;
 
-import org.apache.hudi.client.Compactor;
-import org.apache.hudi.client.HoodieWriteClient;
+import org.apache.hudi.client.AbstractCompactor;
+import org.apache.hudi.client.AbstractHoodieWriteClient;
+import org.apache.hudi.client.common.EngineProperty;
+import org.apache.hudi.client.common.HoodieEngineContext;
 import org.apache.hudi.common.table.timeline.HoodieInstant;
 import org.apache.hudi.common.util.collection.Pair;
 import org.apache.hudi.exception.HoodieIOException;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
-import org.apache.spark.api.java.JavaSparkContext;
 
 import java.io.IOException;
 import java.util.concurrent.BlockingQueue;
@@ -40,7 +41,7 @@
 /**
  * Async Compactor Service that runs in separate thread. Currently, only one compactor is allowed to run at any time.
  */
-public class AsyncCompactService extends AbstractAsyncService {
+public abstract class AsyncCompactService extends HoodieAsyncService {
 
   private static final long serialVersionUID = 1L;
   private static final Logger LOG = LogManager.getLogger(AsyncCompactService.class);
@@ -51,23 +52,25 @@
   public static final String COMPACT_POOL_NAME = "hoodiecompact";
 
   private final int maxConcurrentCompaction;
-  private transient Compactor compactor;
-  private transient JavaSparkContext jssc;
+  private transient AbstractCompactor compactor;
+  private transient HoodieEngineContext context;
   private transient BlockingQueue<HoodieInstant> pendingCompactions = new LinkedBlockingQueue<>();
   private transient ReentrantLock queueLock = new ReentrantLock();
   private transient Condition consumed = queueLock.newCondition();
 
-  public AsyncCompactService(JavaSparkContext jssc, HoodieWriteClient client) {
-    this(jssc, client, false);
+  public AsyncCompactService(HoodieEngineContext context, AbstractHoodieWriteClient client) {
+    this(context, client, false);
   }
 
-  public AsyncCompactService(JavaSparkContext jssc, HoodieWriteClient client, boolean runInDaemonMode) {
+  public AsyncCompactService(HoodieEngineContext context, AbstractHoodieWriteClient client, boolean runInDaemonMode) {
     super(runInDaemonMode);
-    this.jssc = jssc;
-    this.compactor = new Compactor(client);
+    this.context = context;
+    this.compactor = createCompactor(client);
     this.maxConcurrentCompaction = 1;
   }
 
+  protected abstract AbstractCompactor createCompactor(AbstractHoodieWriteClient client);
+
   /**
    * Enqueues new Pending compaction.
    */
@@ -127,8 +130,8 @@
     return Pair.of(CompletableFuture.allOf(IntStream.range(0, maxConcurrentCompaction).mapToObj(i -> CompletableFuture.supplyAsync(() -> {
       try {
         // Set Compactor Pool Name for allowing users to prioritize compaction
-        LOG.info("Setting Spark Pool name for compaction to " + COMPACT_POOL_NAME);
-        jssc.setLocalProperty("spark.scheduler.pool", COMPACT_POOL_NAME);
+        LOG.info("Setting pool name for compaction to " + COMPACT_POOL_NAME);
+        context.setProperty(EngineProperty.COMPACTION_POOL_NAME, COMPACT_POOL_NAME);
 
         while (!isShutdownRequested()) {
           final HoodieInstant instant = fetchNextCompactionInstant();
diff --git a/hudi-client/src/main/java/org/apache/hudi/async/AbstractAsyncService.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/HoodieAsyncService.java
similarity index 94%
rename from hudi-client/src/main/java/org/apache/hudi/async/AbstractAsyncService.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/HoodieAsyncService.java
index 714fa60..32dd042 100644
--- a/hudi-client/src/main/java/org/apache/hudi/async/AbstractAsyncService.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/HoodieAsyncService.java
@@ -34,9 +34,9 @@
 /**
  * Base Class for running clean/delta-sync/compaction in separate thread and controlling their life-cycle.
  */
-public abstract class AbstractAsyncService implements Serializable {
+public abstract class HoodieAsyncService implements Serializable {
 
-  private static final Logger LOG = LogManager.getLogger(AbstractAsyncService.class);
+  private static final Logger LOG = LogManager.getLogger(HoodieAsyncService.class);
 
   // Flag to track if the service is started.
   private boolean started;
@@ -51,11 +51,11 @@
   // Run in daemon mode
   private final boolean runInDaemonMode;
 
-  protected AbstractAsyncService() {
+  protected HoodieAsyncService() {
     this(false);
   }
 
-  protected AbstractAsyncService(boolean runInDaemonMode) {
+  protected HoodieAsyncService(boolean runInDaemonMode) {
     shutdownRequested = false;
     this.runInDaemonMode = runInDaemonMode;
   }
diff --git a/hudi-client/src/main/java/org/apache/hudi/callback/HoodieWriteCommitCallback.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/callback/HoodieWriteCommitCallback.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/callback/HoodieWriteCommitCallback.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/callback/HoodieWriteCommitCallback.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/callback/client/http/HoodieWriteCommitHttpCallbackClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/callback/client/http/HoodieWriteCommitHttpCallbackClient.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/callback/client/http/HoodieWriteCommitHttpCallbackClient.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/callback/client/http/HoodieWriteCommitHttpCallbackClient.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/callback/common/HoodieWriteCommitCallbackMessage.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/callback/common/HoodieWriteCommitCallbackMessage.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/callback/common/HoodieWriteCommitCallbackMessage.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/callback/common/HoodieWriteCommitCallbackMessage.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/callback/impl/HoodieWriteCommitHttpCallback.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/callback/impl/HoodieWriteCommitHttpCallback.java
similarity index 91%
rename from hudi-client/src/main/java/org/apache/hudi/callback/impl/HoodieWriteCommitHttpCallback.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/callback/impl/HoodieWriteCommitHttpCallback.java
index b87ab23..bb60879 100644
--- a/hudi-client/src/main/java/org/apache/hudi/callback/impl/HoodieWriteCommitHttpCallback.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/callback/impl/HoodieWriteCommitHttpCallback.java
@@ -20,13 +20,12 @@
 import org.apache.hudi.callback.HoodieWriteCommitCallback;
 import org.apache.hudi.callback.client.http.HoodieWriteCommitHttpCallbackClient;
 import org.apache.hudi.callback.common.HoodieWriteCommitCallbackMessage;
+import org.apache.hudi.callback.util.HoodieWriteCommitCallbackUtil;
 import org.apache.hudi.config.HoodieWriteConfig;
 
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
 
-import static org.apache.hudi.callback.util.HoodieWriteCommitCallbackUtil.convertToJsonString;
-
 /**
  * A http implementation of {@link HoodieWriteCommitCallback}.
  */
@@ -43,7 +42,7 @@
   @Override
   public void call(HoodieWriteCommitCallbackMessage callbackMessage) {
     // convert to json
-    String callbackMsg = convertToJsonString(callbackMessage);
+    String callbackMsg = HoodieWriteCommitCallbackUtil.convertToJsonString(callbackMessage);
     LOG.info("Try to send callbackMsg, msg = " + callbackMsg);
     client.send(callbackMsg);
   }
diff --git a/hudi-client/src/main/java/org/apache/hudi/callback/util/HoodieCommitCallbackFactory.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/callback/util/HoodieCommitCallbackFactory.java
similarity index 92%
rename from hudi-client/src/main/java/org/apache/hudi/callback/util/HoodieCommitCallbackFactory.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/callback/util/HoodieCommitCallbackFactory.java
index 9d1e9c3..74eb8b6 100644
--- a/hudi-client/src/main/java/org/apache/hudi/callback/util/HoodieCommitCallbackFactory.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/callback/util/HoodieCommitCallbackFactory.java
@@ -20,11 +20,10 @@
 import org.apache.hudi.callback.HoodieWriteCommitCallback;
 import org.apache.hudi.common.util.ReflectionUtils;
 import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.config.HoodieWriteCommitCallbackConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.exception.HoodieCommitCallbackException;
 
-import static org.apache.hudi.config.HoodieWriteCommitCallbackConfig.CALLBACK_CLASS_PROP;
-
 /**
  * Factory help to create {@link HoodieWriteCommitCallback}.
  */
@@ -40,7 +39,7 @@
       return (HoodieWriteCommitCallback) instance;
     } else {
       throw new HoodieCommitCallbackException(String.format("The value of the config option %s can not be null or "
-          + "empty", CALLBACK_CLASS_PROP));
+          + "empty", HoodieWriteCommitCallbackConfig.CALLBACK_CLASS_PROP));
     }
   }
 
diff --git a/hudi-client/src/main/java/org/apache/hudi/callback/util/HoodieWriteCommitCallbackUtil.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/callback/util/HoodieWriteCommitCallbackUtil.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/callback/util/HoodieWriteCommitCallbackUtil.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/callback/util/HoodieWriteCommitCallbackUtil.java
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractCompactor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractCompactor.java
new file mode 100644
index 0000000..30bc569
--- /dev/null
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractCompactor.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.hudi.client;
+
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+
+import java.io.IOException;
+import java.io.Serializable;
+
+/**
+ * Run one round of compaction.
+ */
+public abstract class AbstractCompactor<T extends HoodieRecordPayload, I, K, O> implements Serializable {
+
+  private static final long serialVersionUID = 1L;
+
+  protected transient AbstractHoodieWriteClient<T, I, K, O> compactionClient;
+
+  public AbstractCompactor(AbstractHoodieWriteClient<T, I, K, O> compactionClient) {
+    this.compactionClient = compactionClient;
+  }
+
+  public abstract void compact(HoodieInstant instant) throws IOException;
+}
diff --git a/hudi-client/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java
similarity index 81%
rename from hudi-client/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java
index cdd125e..9628e41 100644
--- a/hudi-client/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieClient.java
@@ -19,17 +19,19 @@
 package org.apache.hudi.client;
 
 import org.apache.hadoop.conf.Configuration;
+
+import org.apache.hudi.client.common.EngineProperty;
 import org.apache.hudi.client.embedded.EmbeddedTimelineService;
-import org.apache.hudi.client.utils.ClientUtils;
+import org.apache.hudi.client.common.HoodieEngineContext;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.config.HoodieWriteConfig;
 
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
-import org.apache.spark.api.java.JavaSparkContext;
 
 import java.io.IOException;
 import java.io.Serializable;
@@ -43,7 +45,7 @@
   private static final Logger LOG = LogManager.getLogger(AbstractHoodieClient.class);
 
   protected final transient FileSystem fs;
-  protected final transient JavaSparkContext jsc;
+  protected final transient HoodieEngineContext context;
   protected final transient Configuration hadoopConf;
   protected final HoodieWriteConfig config;
   protected final String basePath;
@@ -56,15 +58,15 @@
   private transient Option<EmbeddedTimelineService> timelineServer;
   private final boolean shouldStopTimelineServer;
 
-  protected AbstractHoodieClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig) {
-    this(jsc, clientConfig, Option.empty());
+  protected AbstractHoodieClient(HoodieEngineContext context, HoodieWriteConfig clientConfig) {
+    this(context, clientConfig, Option.empty());
   }
 
-  protected AbstractHoodieClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig,
+  protected AbstractHoodieClient(HoodieEngineContext context, HoodieWriteConfig clientConfig,
       Option<EmbeddedTimelineService> timelineServer) {
-    this.hadoopConf = jsc.hadoopConfiguration();
+    this.hadoopConf = context.getHadoopConf().get();
     this.fs = FSUtils.getFs(clientConfig.getBasePath(), hadoopConf);
-    this.jsc = jsc;
+    this.context = context;
     this.basePath = clientConfig.getBasePath();
     this.config = clientConfig;
     this.timelineServer = timelineServer;
@@ -99,7 +101,8 @@
       if (!timelineServer.isPresent()) {
         // Run Embedded Timeline Server
         LOG.info("Starting Timeline service !!");
-        timelineServer = Option.of(new EmbeddedTimelineService(hadoopConf, jsc.getConf(),
+        Option<String> hostAddr = context.getProperty(EngineProperty.EMBEDDED_SERVER_HOST);
+        timelineServer = Option.of(new EmbeddedTimelineService(context, hostAddr.orElse(null),
             config.getClientSpecifiedViewStorageConfig()));
         try {
           timelineServer.get().startServer();
@@ -122,6 +125,8 @@
   }
 
   protected HoodieTableMetaClient createMetaClient(boolean loadActiveTimelineOnLoad) {
-    return ClientUtils.createMetaClient(hadoopConf, config, loadActiveTimelineOnLoad);
+    return new HoodieTableMetaClient(hadoopConf, config.getBasePath(), loadActiveTimelineOnLoad,
+        config.getConsistencyGuardConfig(),
+        Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion())));
   }
 }
diff --git a/hudi-client/src/main/java/org/apache/hudi/client/HoodieWriteClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
similarity index 61%
rename from hudi-client/src/main/java/org/apache/hudi/client/HoodieWriteClient.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
index 7970623..0f35e27 100644
--- a/hudi-client/src/main/java/org/apache/hudi/client/HoodieWriteClient.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
@@ -18,22 +18,27 @@
 
 package org.apache.hudi.client;
 
+import com.codahale.metrics.Timer;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hudi.avro.model.HoodieCleanMetadata;
 import org.apache.hudi.avro.model.HoodieCompactionPlan;
 import org.apache.hudi.avro.model.HoodieRestoreMetadata;
 import org.apache.hudi.avro.model.HoodieRollbackMetadata;
+import org.apache.hudi.callback.HoodieWriteCommitCallback;
+import org.apache.hudi.callback.common.HoodieWriteCommitCallbackMessage;
+import org.apache.hudi.callback.util.HoodieCommitCallbackFactory;
 import org.apache.hudi.client.embedded.EmbeddedTimelineService;
+import org.apache.hudi.client.common.HoodieEngineContext;
 import org.apache.hudi.common.model.HoodieCommitMetadata;
 import org.apache.hudi.common.model.HoodieKey;
-import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.model.HoodieWriteStat;
 import org.apache.hudi.common.model.WriteOperationType;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
 import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
 import org.apache.hudi.common.table.timeline.HoodieInstant;
-import org.apache.hudi.common.table.timeline.HoodieInstant.State;
 import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.CommitUtils;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.ValidationUtils;
 import org.apache.hudi.config.HoodieCompactionConfig;
@@ -45,109 +50,178 @@
 import org.apache.hudi.exception.HoodieSavepointException;
 import org.apache.hudi.index.HoodieIndex;
 import org.apache.hudi.metrics.HoodieMetrics;
+import org.apache.hudi.table.BulkInsertPartitioner;
 import org.apache.hudi.table.HoodieTable;
 import org.apache.hudi.table.HoodieTimelineArchiveLog;
 import org.apache.hudi.table.MarkerFiles;
-import org.apache.hudi.table.BulkInsertPartitioner;
 import org.apache.hudi.table.action.HoodieWriteMetadata;
-import org.apache.hudi.table.action.compact.CompactHelpers;
 import org.apache.hudi.table.action.savepoint.SavepointHelpers;
-
-import com.codahale.metrics.Timer;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
-import org.apache.spark.SparkConf;
-import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
 
 import java.io.IOException;
+import java.nio.charset.StandardCharsets;
 import java.text.ParseException;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.stream.Collectors;
 
 /**
- * Hoodie Write Client helps you build tables on HDFS [insert()] and then perform efficient mutations on an HDFS
- * table [upsert()]
- * <p>
- * Note that, at any given time, there can only be one Spark job performing these operations on a Hoodie table.
+ * Abstract Write Client providing functionality for performing commit, index updates and rollback
+ * Reused for regular write operations like upsert/insert/bulk-insert.. as well as bootstrap
+ *
+ * @param <T> Sub type of HoodieRecordPayload
+ * @param <I> Type of inputs
+ * @param <K> Type of keys
+ * @param <O> Type of outputs
  */
-public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHoodieWriteClient<T> {
+public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I, K, O> extends AbstractHoodieClient {
 
+  protected static final String LOOKUP_STR = "lookup";
   private static final long serialVersionUID = 1L;
-  private static final Logger LOG = LogManager.getLogger(HoodieWriteClient.class);
-  private static final String LOOKUP_STR = "lookup";
-  private final boolean rollbackPending;
-  private final transient HoodieMetrics metrics;
-  private transient Timer.Context compactionTimer;
-  private transient AsyncCleanerService asyncCleanerService;
+  private static final Logger LOG = LogManager.getLogger(AbstractHoodieWriteClient.class);
+
+  protected final transient HoodieMetrics metrics;
+  private final transient HoodieIndex<T, I, K, O> index;
+
+  protected transient Timer.Context writeTimer = null;
+  protected transient Timer.Context compactionTimer;
+
+  private transient WriteOperationType operationType;
+  private transient HoodieWriteCommitCallback commitCallback;
+  protected final boolean rollbackPending;
+  protected transient AsyncCleanerService asyncCleanerService;
 
   /**
    * Create a write client, without cleaning up failed/inflight commits.
    *
-   * @param jsc Java Spark Context
+   * @param context HoodieEngineContext
    * @param clientConfig instance of HoodieWriteConfig
    */
-  public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig) {
-    this(jsc, clientConfig, false);
+  public AbstractHoodieWriteClient(HoodieEngineContext context, HoodieWriteConfig clientConfig) {
+    this(context, clientConfig, false);
   }
 
   /**
    * Create a write client, with new hudi index.
    *
-   * @param jsc Java Spark Context
+   * @param context HoodieEngineContext
    * @param writeConfig instance of HoodieWriteConfig
    * @param rollbackPending whether need to cleanup pending commits
    */
-  public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig writeConfig, boolean rollbackPending) {
-    this(jsc, writeConfig, rollbackPending, HoodieIndex.createIndex(writeConfig));
-  }
-
-  public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig writeConfig, boolean rollbackPending, HoodieIndex index) {
-    this(jsc, writeConfig, rollbackPending, index, Option.empty());
+  public AbstractHoodieWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig, boolean rollbackPending) {
+    this(context, writeConfig, rollbackPending, Option.empty());
   }
 
   /**
-   *  Create a write client, allows to specify all parameters.
+   * Create a write client, allows to specify all parameters.
    *
-   * @param jsc Java Spark Context
+   * @param context         HoodieEngineContext
    * @param writeConfig instance of HoodieWriteConfig
    * @param rollbackPending whether need to cleanup pending commits
    * @param timelineService Timeline Service that runs as part of write client.
    */
-  public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig writeConfig, boolean rollbackPending,
-      HoodieIndex index, Option<EmbeddedTimelineService> timelineService) {
-    super(jsc, index, writeConfig, timelineService);
+  public AbstractHoodieWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig, boolean rollbackPending,
+                                   Option<EmbeddedTimelineService> timelineService) {
+    super(context, writeConfig, timelineService);
     this.metrics = new HoodieMetrics(config, config.getTableName());
     this.rollbackPending = rollbackPending;
+    this.index = createIndex(writeConfig);
+  }
+
+  protected abstract HoodieIndex<T, I, K, O> createIndex(HoodieWriteConfig writeConfig);
+
+  public void setOperationType(WriteOperationType operationType) {
+    this.operationType = operationType;
+  }
+
+  public WriteOperationType getOperationType() {
+    return this.operationType;
   }
 
   /**
-   * Register hudi classes for Kryo serialization.
-   *
-   * @param conf instance of SparkConf
-   * @return SparkConf
+   * Commit changes performed at the given instantTime marker.
    */
-  public static SparkConf registerClasses(SparkConf conf) {
-    conf.registerKryoClasses(new Class[]{HoodieWriteConfig.class, HoodieRecord.class, HoodieKey.class});
-    return conf;
+  public boolean commit(String instantTime, O writeStatuses) {
+    return commit(instantTime, writeStatuses, Option.empty());
+  }
+
+  /**
+   *
+   * Commit changes performed at the given instantTime marker.
+   */
+  public boolean commit(String instantTime, O writeStatuses, Option<Map<String, String>> extraMetadata) {
+    HoodieTableMetaClient metaClient = createMetaClient(false);
+    String actionType = metaClient.getCommitActionType();
+    return commit(instantTime, writeStatuses, extraMetadata, actionType, Collections.emptyMap());
+  }
+
+  public abstract boolean commit(String instantTime, O writeStatuses, Option<Map<String, String>> extraMetadata,
+                                 String commitActionType, Map<String, List<String>> partitionToReplacedFileIds);
+
+  public boolean commitStats(String instantTime, List<HoodieWriteStat> stats, Option<Map<String, String>> extraMetadata,
+                             String commitActionType) {
+    return commitStats(instantTime, stats, extraMetadata, commitActionType, Collections.emptyMap());
+  }
+
+  public boolean commitStats(String instantTime, List<HoodieWriteStat> stats, Option<Map<String, String>> extraMetadata,
+                             String commitActionType, Map<String, List<String>> partitionToReplaceFileIds) {
+    LOG.info("Committing " + instantTime + " action " + commitActionType);
+    // Create a Hoodie table which encapsulated the commits and files visible
+    HoodieTable table = createTable(config, hadoopConf);
+
+    HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
+    HoodieCommitMetadata metadata = CommitUtils.buildMetadata(stats, partitionToReplaceFileIds, extraMetadata, operationType, config.getSchema(), commitActionType);
+    // Finalize write
+    finalizeWrite(table, instantTime, stats);
+
+    try {
+      activeTimeline.saveAsComplete(new HoodieInstant(true, commitActionType, instantTime),
+          Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
+      postCommit(table, metadata, instantTime, extraMetadata);
+      emitCommitMetrics(instantTime, metadata, commitActionType);
+      LOG.info("Committed " + instantTime);
+    } catch (IOException e) {
+      throw new HoodieCommitException("Failed to complete commit " + config.getBasePath() + " at time " + instantTime,
+          e);
+    }
+
+    // callback if needed.
+    if (config.writeCommitCallbackOn()) {
+      if (null == commitCallback) {
+        commitCallback = HoodieCommitCallbackFactory.create(config);
+      }
+      commitCallback.call(new HoodieWriteCommitCallbackMessage(instantTime, config.getTableName(), config.getBasePath()));
+    }
+    return true;
+  }
+
+  protected abstract HoodieTable<T, I, K, O> createTable(HoodieWriteConfig config, Configuration hadoopConf);
+
+  void emitCommitMetrics(String instantTime, HoodieCommitMetadata metadata, String actionType) {
+    try {
+
+      if (writeTimer != null) {
+        long durationInMs = metrics.getDurationInMs(writeTimer.stop());
+        metrics.updateCommitMetrics(HoodieActiveTimeline.COMMIT_FORMATTER.parse(instantTime).getTime(), durationInMs,
+            metadata, actionType);
+        writeTimer = null;
+      }
+    } catch (ParseException e) {
+      throw new HoodieCommitException("Failed to complete commit " + config.getBasePath() + " at time " + instantTime
+          + "Instant time is not of valid format", e);
+    }
   }
 
   /**
    * Filter out HoodieRecords that already exists in the output folder. This is useful in deduplication.
    *
-   * @param hoodieRecords Input RDD of Hoodie records.
-   * @return A subset of hoodieRecords RDD, with existing records filtered out.
+   * @param hoodieRecords Input Hoodie records.
+   * @return A subset of hoodieRecords, with existing records filtered out.
    */
-  public JavaRDD<HoodieRecord<T>> filterExists(JavaRDD<HoodieRecord<T>> hoodieRecords) {
-    // Create a Hoodie table which encapsulated the commits and files visible
-    HoodieTable<T> table = HoodieTable.create(config, hadoopConf);
-    Timer.Context indexTimer = metrics.getIndexCtx();
-    JavaRDD<HoodieRecord<T>> recordsWithLocation = getIndex().tagLocation(hoodieRecords, jsc, table);
-    metrics.updateIndexMetrics(LOOKUP_STR, metrics.getDurationInMs(indexTimer == null ? 0L : indexTimer.stop()));
-    return recordsWithLocation.filter(v1 -> !v1.isCurrentLocationKnown());
-  }
+  public abstract I filterExists(I hoodieRecords);
 
   /**
    * Main API to run bootstrap to hudi.
@@ -156,8 +230,8 @@
     if (rollbackPending) {
       rollBackInflightBootstrap();
     }
-    HoodieTable<T> table = getTableAndInitCtx(WriteOperationType.UPSERT, HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS);
-    table.bootstrap(jsc, extraMetadata);
+    HoodieTable<T, I, K, O> table = getTableAndInitCtx(WriteOperationType.UPSERT, HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS);
+    table.bootstrap(context, extraMetadata);
   }
 
   /**
@@ -165,14 +239,14 @@
    */
   protected void rollBackInflightBootstrap() {
     LOG.info("Rolling back pending bootstrap if present");
-    HoodieTable<T> table = HoodieTable.create(config, hadoopConf);
+    HoodieTable<T, I, K, O> table = createTable(config, hadoopConf);
     HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompaction();
     Option<String> instant = Option.fromJavaOptional(
         inflightTimeline.getReverseOrderedInstants().map(HoodieInstant::getTimestamp).findFirst());
     if (instant.isPresent() && HoodieTimeline.compareTimestamps(instant.get(), HoodieTimeline.LESSER_THAN_OR_EQUALS,
         HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS)) {
       LOG.info("Found pending bootstrap instants. Rolling them back");
-      table.rollbackBootstrap(jsc, HoodieActiveTimeline.createNewInstantTime());
+      table.rollbackBootstrap(context, HoodieActiveTimeline.createNewInstantTime());
       LOG.info("Finished rolling back pending bootstrap");
     }
 
@@ -181,21 +255,11 @@
   /**
    * Upsert a batch of new records into Hoodie table at the supplied instantTime.
    *
-   * @param records JavaRDD of hoodieRecords to upsert
+   * @param records hoodieRecords to upsert
    * @param instantTime Instant time of the commit
-   * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
+   * @return WriteStatus to inspect errors and counts
    */
-  public JavaRDD<WriteStatus> upsert(JavaRDD<HoodieRecord<T>> records, final String instantTime) {
-    HoodieTable<T> table = getTableAndInitCtx(WriteOperationType.UPSERT, instantTime);
-    table.validateUpsertSchema();
-    setOperationType(WriteOperationType.UPSERT);
-    this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime);
-    HoodieWriteMetadata result = table.upsert(jsc, instantTime, records);
-    if (result.getIndexLookupDuration().isPresent()) {
-      metrics.updateIndexMetrics(LOOKUP_STR, result.getIndexLookupDuration().get().toMillis());
-    }
-    return postWrite(result, instantTime, table);
-  }
+  public abstract O upsert(I records, final String instantTime);
 
   /**
    * Upserts the given prepared records into the Hoodie table, at the supplied instantTime.
@@ -206,14 +270,7 @@
    * @param instantTime Instant time of the commit
    * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
    */
-  public JavaRDD<WriteStatus> upsertPreppedRecords(JavaRDD<HoodieRecord<T>> preppedRecords, final String instantTime) {
-    HoodieTable<T> table = getTableAndInitCtx(WriteOperationType.UPSERT_PREPPED, instantTime);
-    table.validateUpsertSchema();
-    setOperationType(WriteOperationType.UPSERT_PREPPED);
-    this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime);
-    HoodieWriteMetadata result = table.upsertPrepped(jsc,instantTime, preppedRecords);
-    return postWrite(result, instantTime, table);
-  }
+  public abstract O upsertPreppedRecords(I preppedRecords, final String instantTime);
 
   /**
    * Inserts the given HoodieRecords, into the table. This API is intended to be used for normal writes.
@@ -225,14 +282,7 @@
    * @param instantTime Instant time of the commit
    * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
    */
-  public JavaRDD<WriteStatus> insert(JavaRDD<HoodieRecord<T>> records, final String instantTime) {
-    HoodieTable<T> table = getTableAndInitCtx(WriteOperationType.INSERT, instantTime);
-    table.validateInsertSchema();
-    setOperationType(WriteOperationType.INSERT);
-    this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime);
-    HoodieWriteMetadata result = table.insert(jsc,instantTime, records);
-    return postWrite(result, instantTime, table);
-  }
+  public abstract O insert(I records, final String instantTime);
 
   /**
    * Inserts the given prepared records into the Hoodie table, at the supplied instantTime.
@@ -245,36 +295,27 @@
    * @param instantTime Instant time of the commit
    * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
    */
-  public JavaRDD<WriteStatus> insertPreppedRecords(JavaRDD<HoodieRecord<T>> preppedRecords, final String instantTime) {
-    HoodieTable<T> table = getTableAndInitCtx(WriteOperationType.INSERT_PREPPED, instantTime);
-    table.validateInsertSchema();
-    setOperationType(WriteOperationType.INSERT_PREPPED);
-    this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime);
-    HoodieWriteMetadata result = table.insertPrepped(jsc,instantTime, preppedRecords);
-    return postWrite(result, instantTime, table);
-  }
+  public abstract O insertPreppedRecords(I preppedRecords, final String instantTime);
 
   /**
    * Loads the given HoodieRecords, as inserts into the table. This is suitable for doing big bulk loads into a Hoodie
    * table for the very first time (e.g: converting an existing table to Hoodie).
    * <p>
    * This implementation uses sortBy (which does range partitioning based on reservoir sampling) and attempts to control
-   * the numbers of files with less memory compared to the {@link HoodieWriteClient#insert(JavaRDD, String)}
+   * the numbers of files with less memory compared to the {@link AbstractHoodieWriteClient#insert(I, String)}
    *
    * @param records HoodieRecords to insert
    * @param instantTime Instant time of the commit
    * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
    */
-  public JavaRDD<WriteStatus> bulkInsert(JavaRDD<HoodieRecord<T>> records, final String instantTime) {
-    return bulkInsert(records, instantTime, Option.empty());
-  }
+  public abstract O bulkInsert(I records, final String instantTime);
 
   /**
    * Loads the given HoodieRecords, as inserts into the table. This is suitable for doing big bulk loads into a Hoodie
    * table for the very first time (e.g: converting an existing table to Hoodie).
    * <p>
    * This implementation uses sortBy (which does range partitioning based on reservoir sampling) and attempts to control
-   * the numbers of files with less memory compared to the {@link HoodieWriteClient#insert(JavaRDD, String)}. Optionally
+   * the numbers of files with less memory compared to the {@link AbstractHoodieWriteClient#insert(I, String)}. Optionally
    * it allows users to specify their own partitioner. If specified then it will be used for repartitioning records. See
    * {@link BulkInsertPartitioner}.
    *
@@ -284,15 +325,9 @@
    * into hoodie.
    * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
    */
-  public JavaRDD<WriteStatus> bulkInsert(JavaRDD<HoodieRecord<T>> records, final String instantTime,
-                                         Option<BulkInsertPartitioner> userDefinedBulkInsertPartitioner) {
-    HoodieTable<T> table = getTableAndInitCtx(WriteOperationType.BULK_INSERT, instantTime);
-    table.validateInsertSchema();
-    setOperationType(WriteOperationType.BULK_INSERT);
-    this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime);
-    HoodieWriteMetadata result = table.bulkInsert(jsc,instantTime, records, userDefinedBulkInsertPartitioner);
-    return postWrite(result, instantTime, table);
-  }
+  public abstract O bulkInsert(I records, final String instantTime,
+                               Option<BulkInsertPartitioner<I>> userDefinedBulkInsertPartitioner);
+
 
   /**
    * Loads the given HoodieRecords, as inserts into the table. This is suitable for doing big bulk loads into a Hoodie
@@ -300,7 +335,7 @@
    * duplicates if needed.
    * <p>
    * This implementation uses sortBy (which does range partitioning based on reservoir sampling) and attempts to control
-   * the numbers of files with less memory compared to the {@link HoodieWriteClient#insert(JavaRDD, String)}. Optionally
+   * the numbers of files with less memory compared to the {@link AbstractHoodieWriteClient#insert(I, String)}. Optionally
    * it allows users to specify their own partitioner. If specified then it will be used for repartitioning records. See
    * {@link BulkInsertPartitioner}.
    *
@@ -310,31 +345,8 @@
    * into hoodie.
    * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
    */
-  public JavaRDD<WriteStatus> bulkInsertPreppedRecords(JavaRDD<HoodieRecord<T>> preppedRecords, final String instantTime,
-                                                       Option<BulkInsertPartitioner> bulkInsertPartitioner) {
-    HoodieTable<T> table = getTableAndInitCtx(WriteOperationType.BULK_INSERT_PREPPED, instantTime);
-    table.validateInsertSchema();
-    setOperationType(WriteOperationType.BULK_INSERT_PREPPED);
-    this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime);
-    HoodieWriteMetadata result = table.bulkInsertPrepped(jsc,instantTime, preppedRecords, bulkInsertPartitioner);
-    return postWrite(result, instantTime, table);
-  }
-
-  /**
-   * Removes all existing records from the partitions affected and inserts the given HoodieRecords, into the table.
-
-   * @param records HoodieRecords to insert
-   * @param instantTime Instant time of the commit
-   * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
-   */
-  public HoodieWriteResult insertOverwrite(JavaRDD<HoodieRecord<T>> records, final String instantTime) {
-    HoodieTable<T> table = getTableAndInitCtx(WriteOperationType.INSERT_OVERWRITE, instantTime);
-    table.validateInsertSchema();
-    setOperationType(WriteOperationType.INSERT_OVERWRITE);
-    this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime);
-    HoodieWriteMetadata result = table.insertOverwrite(jsc, instantTime, records);
-    return new HoodieWriteResult(postWrite(result, instantTime, table), result.getPartitionToReplaceFileIds());
-  }
+  public abstract O bulkInsertPreppedRecords(I preppedRecords, final String instantTime,
+                                             Option<BulkInsertPartitioner<I>> bulkInsertPartitioner);
 
   /**
    * Deletes a list of {@link HoodieKey}s from the Hoodie table, at the supplied instantTime {@link HoodieKey}s will be
@@ -344,12 +356,7 @@
    * @param instantTime Commit time handle
    * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
    */
-  public JavaRDD<WriteStatus> delete(JavaRDD<HoodieKey> keys, final String instantTime) {
-    HoodieTable<T> table = getTableAndInitCtx(WriteOperationType.DELETE, instantTime);
-    setOperationType(WriteOperationType.DELETE);
-    HoodieWriteMetadata result = table.delete(jsc,instantTime, keys);
-    return postWrite(result, instantTime, table);
-  }
+  public abstract O delete(K keys, final String instantTime);
 
   /**
    * Common method containing steps to be performed after write (upsert/insert/..) operations including auto-commit.
@@ -358,31 +365,21 @@
    * @param hoodieTable Hoodie Table
    * @return Write Status
    */
-  private JavaRDD<WriteStatus> postWrite(HoodieWriteMetadata result, String instantTime, HoodieTable<T> hoodieTable) {
-    if (result.getIndexLookupDuration().isPresent()) {
-      metrics.updateIndexMetrics(getOperationType().name(), result.getIndexUpdateDuration().get().toMillis());
-    }
-    if (result.isCommitted()) {
-      // Perform post commit operations.
-      if (result.getFinalizeDuration().isPresent()) {
-        metrics.updateFinalizeWriteMetrics(result.getFinalizeDuration().get().toMillis(),
-            result.getWriteStats().get().size());
-      }
+  protected abstract O postWrite(HoodieWriteMetadata<O> result, String instantTime, HoodieTable<T, I, K, O> hoodieTable);
 
-      postCommit(hoodieTable, result.getCommitMetadata().get(), instantTime, Option.empty());
-
-      emitCommitMetrics(instantTime, result.getCommitMetadata().get(),
-          hoodieTable.getMetaClient().getCommitActionType());
-    }
-    return result.getWriteStatuses();
-  }
-
-  @Override
-  protected void postCommit(HoodieTable<?> table, HoodieCommitMetadata metadata, String instantTime, Option<Map<String, String>> extraMetadata) {
+  /**
+   * Post Commit Hook. Derived classes use this method to perform post-commit processing
+   *
+   * @param table         table to commit on
+   * @param metadata      Commit Metadata corresponding to committed instant
+   * @param instantTime   Instant Time
+   * @param extraMetadata Additional Metadata passed by user
+   */
+  protected void postCommit(HoodieTable<T, I, K, O> table, HoodieCommitMetadata metadata, String instantTime, Option<Map<String, String>> extraMetadata) {
     try {
 
       // Delete the marker directory for the instant.
-      new MarkerFiles(table, instantTime).quietDeleteMarkerDir(jsc, config.getMarkersDeleteParallelism());
+      new MarkerFiles(table, instantTime).quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism());
 
       // Do an inline compaction if enabled
       if (config.isInlineCompaction()) {
@@ -393,15 +390,15 @@
         metadata.addMetadata(HoodieCompactionConfig.INLINE_COMPACT_PROP, "false");
       }
       // We cannot have unbounded commit files. Archive commits if we have to archive
-      HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(config, hadoopConf);
-      archiveLog.archiveIfRequired(jsc);
+      HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(config, table);
+      archiveLog.archiveIfRequired(context);
       autoCleanOnCommit(instantTime);
     } catch (IOException ioe) {
       throw new HoodieIOException(ioe.getMessage(), ioe);
     }
   }
 
-  private void runAnyPendingCompactions(HoodieTable<?> table) {
+  protected void runAnyPendingCompactions(HoodieTable<T, I, K, O> table) {
     table.getActiveTimeline().getCommitsAndCompactionTimeline().filterPendingCompactionTimeline().getInstants()
         .forEach(instant -> {
           LOG.info("Running previously failed inflight compaction at instant " + instant);
@@ -411,9 +408,10 @@
 
   /**
    * Handle auto clean during commit.
+   *
    * @param instantTime
    */
-  private void autoCleanOnCommit(String instantTime) {
+  protected void autoCleanOnCommit(String instantTime) {
     if (config.isAutoClean()) {
       // Call clean to cleanup if there is anything to cleanup after the commit,
       if (config.isAsyncClean()) {
@@ -434,7 +432,7 @@
    * @param comment - Comment for the savepoint
    */
   public void savepoint(String user, String comment) {
-    HoodieTable<T> table = HoodieTable.create(config, hadoopConf);
+    HoodieTable<T, I, K, O> table = createTable(config, hadoopConf);
     if (table.getCompletedCommitsTimeline().empty()) {
       throw new HoodieSavepointException("Could not savepoint. Commit timeline is empty");
     }
@@ -458,8 +456,8 @@
    * @param comment - Comment for the savepoint
    */
   public void savepoint(String instantTime, String user, String comment) {
-    HoodieTable<T> table = HoodieTable.create(config, hadoopConf);
-    table.savepoint(jsc, instantTime, user, comment);
+    HoodieTable<T, I, K, O> table = createTable(config, hadoopConf);
+    table.savepoint(context, instantTime, user, comment);
   }
 
   /**
@@ -470,7 +468,7 @@
    * @return true if the savepoint was deleted successfully
    */
   public void deleteSavepoint(String savepointTime) {
-    HoodieTable<T> table = HoodieTable.create(config, hadoopConf);
+    HoodieTable<T, I, K, O> table = createTable(config, hadoopConf);
     SavepointHelpers.deleteSavepoint(table, savepointTime);
   }
 
@@ -485,7 +483,7 @@
    * @return true if the savepoint was restored to successfully
    */
   public void restoreToSavepoint(String savepointTime) {
-    HoodieTable<T> table = HoodieTable.create(config, hadoopConf);
+    HoodieTable<T, I, K, O> table = createTable(config, hadoopConf);
     SavepointHelpers.validateSavepointPresence(table, savepointTime);
     restoreToInstant(savepointTime);
     SavepointHelpers.validateSavepointRestore(table, savepointTime);
@@ -500,16 +498,16 @@
   public boolean rollback(final String commitInstantTime) throws HoodieRollbackException {
     LOG.info("Begin rollback of instant " + commitInstantTime);
     final String rollbackInstantTime = HoodieActiveTimeline.createNewInstantTime();
-    final Timer.Context context = this.metrics.getRollbackCtx();
+    final Timer.Context timerContext = this.metrics.getRollbackCtx();
     try {
-      HoodieTable<T> table = HoodieTable.create(config, hadoopConf);
+      HoodieTable<T, I, K, O> table = createTable(config, hadoopConf);
       Option<HoodieInstant> commitInstantOpt = Option.fromJavaOptional(table.getActiveTimeline().getCommitsTimeline().getInstants()
-              .filter(instant -> HoodieActiveTimeline.EQUALS.test(instant.getTimestamp(), commitInstantTime))
-              .findFirst());
+          .filter(instant -> HoodieActiveTimeline.EQUALS.test(instant.getTimestamp(), commitInstantTime))
+          .findFirst());
       if (commitInstantOpt.isPresent()) {
-        HoodieRollbackMetadata rollbackMetadata = table.rollback(jsc, rollbackInstantTime, commitInstantOpt.get(), true);
-        if (context != null) {
-          long durationInMs = metrics.getDurationInMs(context.stop());
+        HoodieRollbackMetadata rollbackMetadata = table.rollback(context, rollbackInstantTime, commitInstantOpt.get(), true);
+        if (timerContext != null) {
+          long durationInMs = metrics.getDurationInMs(timerContext.stop());
           metrics.updateRollbackMetrics(durationInMs, rollbackMetadata.getTotalFilesDeleted());
         }
         return true;
@@ -531,12 +529,12 @@
   public HoodieRestoreMetadata restoreToInstant(final String instantTime) throws HoodieRestoreException {
     LOG.info("Begin restore to instant " + instantTime);
     final String restoreInstantTime = HoodieActiveTimeline.createNewInstantTime();
-    Timer.Context context = metrics.getRollbackCtx();
+    Timer.Context timerContext = metrics.getRollbackCtx();
     try {
-      HoodieTable<T> table = HoodieTable.create(config, hadoopConf);
-      HoodieRestoreMetadata restoreMetadata = table.restore(jsc, restoreInstantTime, instantTime);
-      if (context != null) {
-        final long durationInMs = metrics.getDurationInMs(context.stop());
+      HoodieTable<T, I, K, O> table = createTable(config, hadoopConf);
+      HoodieRestoreMetadata restoreMetadata = table.restore(context, restoreInstantTime, instantTime);
+      if (timerContext != null) {
+        final long durationInMs = metrics.getDurationInMs(timerContext.stop());
         final long totalFilesDeleted = restoreMetadata.getHoodieRestoreMetadata().values().stream()
             .flatMap(Collection::stream)
             .mapToLong(HoodieRollbackMetadata::getTotalFilesDeleted)
@@ -550,26 +548,16 @@
   }
 
   /**
-   * Releases any resources used by the client.
-   */
-  @Override
-  public void close() {
-    AsyncCleanerService.forceShutdown(asyncCleanerService);
-    asyncCleanerService = null;
-    super.close();
-  }
-
-  /**
    * Clean up any stale/old files/data lying around (either on file storage or index storage) based on the
    * configurations and CleaningPolicy used. (typically files that no longer can be used by a running query can be
    * cleaned)
    */
   public HoodieCleanMetadata clean(String cleanInstantTime) throws HoodieIOException {
     LOG.info("Cleaner started");
-    final Timer.Context context = metrics.getCleanCtx();
-    HoodieCleanMetadata metadata = HoodieTable.create(config, hadoopConf).clean(jsc, cleanInstantTime);
-    if (context != null && metadata != null) {
-      long durationMs = metrics.getDurationInMs(context.stop());
+    final Timer.Context timerContext = metrics.getCleanCtx();
+    HoodieCleanMetadata metadata = createTable(config, hadoopConf).clean(context, cleanInstantTime);
+    if (timerContext != null && metadata != null) {
+      long durationMs = metrics.getDurationInMs(timerContext.stop());
       metrics.updateCleanMetrics(durationMs, metadata.getTotalFilesDeleted());
       LOG.info("Cleaned " + metadata.getTotalFilesDeleted() + " files"
           + " Earliest Retained Instant :" + metadata.getEarliestCommitToRetain()
@@ -634,7 +622,7 @@
             HoodieTimeline.compareTimestamps(latestPending.getTimestamp(), HoodieTimeline.LESSER_THAN, instantTime),
         "Latest pending compaction instant time must be earlier than this instant time. Latest Compaction :"
             + latestPending + ",  Ingesting at " + instantTime));
-    metaClient.getActiveTimeline().createNewInstant(new HoodieInstant(State.REQUESTED, actionType,
+    metaClient.getActiveTimeline().createNewInstant(new HoodieInstant(HoodieInstant.State.REQUESTED, actionType,
         instantTime));
   }
 
@@ -656,8 +644,8 @@
    */
   public boolean scheduleCompactionAtInstant(String instantTime, Option<Map<String, String>> extraMetadata) throws HoodieIOException {
     LOG.info("Scheduling compaction at instant time :" + instantTime);
-    Option<HoodieCompactionPlan> plan = HoodieTable.create(config, hadoopConf)
-        .scheduleCompaction(jsc, instantTime, extraMetadata);
+    Option<HoodieCompactionPlan> plan = createTable(config, hadoopConf)
+        .scheduleCompaction(context, instantTime, extraMetadata);
     return plan.isPresent();
   }
 
@@ -667,7 +655,7 @@
    * @param compactionInstantTime Compaction Instant Time
    * @return RDD of WriteStatus to inspect errors and counts
    */
-  public JavaRDD<WriteStatus> compact(String compactionInstantTime) {
+  public O compact(String compactionInstantTime) {
     return compact(compactionInstantTime, config.shouldAutoCommit());
   }
 
@@ -678,38 +666,15 @@
    * @param writeStatuses RDD of WriteStatus to inspect errors and counts
    * @param extraMetadata Extra Metadata to be stored
    */
-  public void commitCompaction(String compactionInstantTime, JavaRDD<WriteStatus> writeStatuses,
-                               Option<Map<String, String>> extraMetadata) throws IOException {
-    HoodieTable<T> table = HoodieTable.create(config, hadoopConf);
-    HoodieCommitMetadata metadata = CompactHelpers.createCompactionMetadata(
-        table, compactionInstantTime, writeStatuses, config.getSchema());
-    extraMetadata.ifPresent(m -> m.forEach(metadata::addMetadata));
-    completeCompaction(metadata, writeStatuses, table, compactionInstantTime);
-  }
+  public abstract void commitCompaction(String compactionInstantTime, O writeStatuses,
+                                        Option<Map<String, String>> extraMetadata) throws IOException;
 
   /**
    * Commit Compaction and track metrics.
    */
-  protected void completeCompaction(HoodieCommitMetadata metadata, JavaRDD<WriteStatus> writeStatuses, HoodieTable<T> table,
-                                    String compactionCommitTime) {
+  protected abstract void completeCompaction(HoodieCommitMetadata metadata, O writeStatuses,
+                                             HoodieTable<T, I, K, O> table, String compactionCommitTime);
 
-    List<HoodieWriteStat> writeStats = writeStatuses.map(WriteStatus::getStat).collect();
-    finalizeWrite(table, compactionCommitTime, writeStats);
-    LOG.info("Committing Compaction " + compactionCommitTime + ". Finished with result " + metadata);
-    CompactHelpers.completeInflightCompaction(table, compactionCommitTime, metadata);
-
-    if (compactionTimer != null) {
-      long durationInMs = metrics.getDurationInMs(compactionTimer.stop());
-      try {
-        metrics.updateCommitMetrics(HoodieActiveTimeline.COMMIT_FORMATTER.parse(compactionCommitTime).getTime(),
-            durationInMs, metadata, HoodieActiveTimeline.COMPACTION_ACTION);
-      } catch (ParseException e) {
-        throw new HoodieCommitException("Commit time is not of valid format. Failed to commit compaction "
-            + config.getBasePath() + " at time " + compactionCommitTime, e);
-      }
-    }
-    LOG.info("Compacted successfully on commit " + compactionCommitTime);
-  }
 
   /**
    * Rollback failed compactions. Inflight rollbacks for compactions revert the .inflight file to the .requested file
@@ -717,8 +682,8 @@
    * @param inflightInstant Inflight Compaction Instant
    * @param table Hoodie Table
    */
-  public void rollbackInflightCompaction(HoodieInstant inflightInstant, HoodieTable table) {
-    table.rollback(jsc, HoodieActiveTimeline.createNewInstantTime(), inflightInstant, false);
+  public void rollbackInflightCompaction(HoodieInstant inflightInstant, HoodieTable<T, I, K, O> table) {
+    table.rollback(context, HoodieActiveTimeline.createNewInstantTime(), inflightInstant, false);
     table.getActiveTimeline().revertCompactionInflightToRequested(inflightInstant);
   }
 
@@ -726,7 +691,7 @@
    * Cleanup all pending commits.
    */
   private void rollbackPendingCommits() {
-    HoodieTable<T> table = HoodieTable.create(config, hadoopConf);
+    HoodieTable<T, I, K, O> table = createTable(config, hadoopConf);
     HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompaction();
     List<String> commits = inflightTimeline.getReverseOrderedInstants().map(HoodieInstant::getTimestamp)
         .collect(Collectors.toList());
@@ -747,27 +712,12 @@
    * @param compactionInstantTime Compaction Instant Time
    * @return RDD of Write Status
    */
-  private JavaRDD<WriteStatus> compact(String compactionInstantTime, boolean shouldComplete) {
-    HoodieTable<T> table = HoodieTable.create(config, hadoopConf);
-    HoodieTimeline pendingCompactionTimeline = table.getActiveTimeline().filterPendingCompactionTimeline();
-    HoodieInstant inflightInstant = HoodieTimeline.getCompactionInflightInstant(compactionInstantTime);
-    if (pendingCompactionTimeline.containsInstant(inflightInstant)) {
-      rollbackInflightCompaction(inflightInstant, table);
-      table.getMetaClient().reloadActiveTimeline();
-    }
-    compactionTimer = metrics.getCompactionCtx();
-    HoodieWriteMetadata compactionMetadata = table.compact(jsc, compactionInstantTime);
-    JavaRDD<WriteStatus> statuses = compactionMetadata.getWriteStatuses();
-    if (shouldComplete && compactionMetadata.getCommitMetadata().isPresent()) {
-      completeCompaction(compactionMetadata.getCommitMetadata().get(), statuses, table, compactionInstantTime);
-    }
-    return statuses;
-  }
+  protected abstract O compact(String compactionInstantTime, boolean shouldComplete);
 
   /**
    * Performs a compaction operation on a table, serially before or after an insert/upsert action.
    */
-  private Option<String> inlineCompact(Option<Map<String, String>> extraMetadata) {
+  protected Option<String> inlineCompact(Option<Map<String, String>> extraMetadata) {
     Option<String> compactionInstantTimeOpt = scheduleCompaction(extraMetadata);
     compactionInstantTimeOpt.ifPresent(compactionInstantTime -> {
       // inline compaction should auto commit as the user is never given control
@@ -775,4 +725,82 @@
     });
     return compactionInstantTimeOpt;
   }
+
+  /**
+   * Finalize Write operation.
+   *
+   * @param table HoodieTable
+   * @param instantTime Instant Time
+   * @param stats Hoodie Write Stat
+   */
+  protected void finalizeWrite(HoodieTable<T, I, K, O> table, String instantTime, List<HoodieWriteStat> stats) {
+    try {
+      final Timer.Context finalizeCtx = metrics.getFinalizeCtx();
+      table.finalizeWrite(context, instantTime, stats);
+      if (finalizeCtx != null) {
+        Option<Long> durationInMs = Option.of(metrics.getDurationInMs(finalizeCtx.stop()));
+        durationInMs.ifPresent(duration -> {
+          LOG.info("Finalize write elapsed time (milliseconds): " + duration);
+          metrics.updateFinalizeWriteMetrics(duration, stats.size());
+        });
+      }
+    } catch (HoodieIOException ioe) {
+      throw new HoodieCommitException("Failed to complete commit " + instantTime + " due to finalize errors.", ioe);
+    }
+  }
+
+  public HoodieMetrics getMetrics() {
+    return metrics;
+  }
+
+  public HoodieIndex<T, I, K, O> getIndex() {
+    return index;
+  }
+
+  /**
+   * Get HoodieTable and init {@link Timer.Context}.
+   *
+   * @param operationType write operation type
+   * @param instantTime current inflight instant time
+   * @return HoodieTable
+   */
+  protected abstract HoodieTable<T, I, K, O> getTableAndInitCtx(WriteOperationType operationType, String instantTime);
+
+  /**
+   * Sets write schema from last instant since deletes may not have schema set in the config.
+   */
+  protected void setWriteSchemaForDeletes(HoodieTableMetaClient metaClient) {
+    try {
+      HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline();
+      Option<HoodieInstant> lastInstant =
+          activeTimeline.filterCompletedInstants().filter(s -> s.getAction().equals(metaClient.getCommitActionType()))
+              .lastInstant();
+      if (lastInstant.isPresent()) {
+        HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(
+            activeTimeline.getInstantDetails(lastInstant.get()).get(), HoodieCommitMetadata.class);
+        if (commitMetadata.getExtraMetadata().containsKey(HoodieCommitMetadata.SCHEMA_KEY)) {
+          config.setSchema(commitMetadata.getExtraMetadata().get(HoodieCommitMetadata.SCHEMA_KEY));
+        } else {
+          throw new HoodieIOException("Latest commit does not have any schema in commit metadata");
+        }
+      } else {
+        throw new HoodieIOException("Deletes issued without any prior commits");
+      }
+    } catch (IOException e) {
+      throw new HoodieIOException("IOException thrown while reading last commit metadata", e);
+    }
+  }
+
+  @Override
+  public void close() {
+    // release AsyncCleanerService
+    AsyncCleanerService.forceShutdown(asyncCleanerService);
+    asyncCleanerService = null;
+
+    // Stop timeline-server if running
+    super.close();
+    // Calling this here releases any resources used by your index, so make sure to finish any related operations
+    // before this point
+    this.index.close();
+  }
 }
diff --git a/hudi-client/src/main/java/org/apache/hudi/client/AsyncCleanerService.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AsyncCleanerService.java
similarity index 90%
rename from hudi-client/src/main/java/org/apache/hudi/client/AsyncCleanerService.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AsyncCleanerService.java
index 6367e79..e8016c9 100644
--- a/hudi-client/src/main/java/org/apache/hudi/client/AsyncCleanerService.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AsyncCleanerService.java
@@ -18,7 +18,7 @@
 
 package org.apache.hudi.client;
 
-import org.apache.hudi.async.AbstractAsyncService;
+import org.apache.hudi.async.HoodieAsyncService;
 import org.apache.hudi.common.util.collection.Pair;
 import org.apache.hudi.exception.HoodieException;
 import org.apache.log4j.LogManager;
@@ -31,15 +31,15 @@
 /**
  * Clean service running concurrently with write operation.
  */
-class AsyncCleanerService extends AbstractAsyncService {
+class AsyncCleanerService extends HoodieAsyncService {
 
   private static final Logger LOG = LogManager.getLogger(AsyncCleanerService.class);
 
-  private final HoodieWriteClient<?> writeClient;
+  private final AbstractHoodieWriteClient writeClient;
   private final String cleanInstantTime;
   private final transient ExecutorService executor = Executors.newSingleThreadExecutor();
 
-  protected AsyncCleanerService(HoodieWriteClient<?> writeClient, String cleanInstantTime) {
+  protected AsyncCleanerService(AbstractHoodieWriteClient writeClient, String cleanInstantTime) {
     this.writeClient = writeClient;
     this.cleanInstantTime = cleanInstantTime;
   }
@@ -52,7 +52,7 @@
     }), executor);
   }
 
-  public static AsyncCleanerService startAsyncCleaningIfEnabled(HoodieWriteClient writeClient,
+  public static AsyncCleanerService startAsyncCleaningIfEnabled(AbstractHoodieWriteClient writeClient,
                                                                 String instantTime) {
     AsyncCleanerService asyncCleanerService = null;
     if (writeClient.getConfig().isAutoClean() && writeClient.getConfig().isAsyncClean()) {
diff --git a/hudi-client/src/main/java/org/apache/hudi/client/CompactionAdminClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/CompactionAdminClient.java
similarity index 96%
rename from hudi-client/src/main/java/org/apache/hudi/client/CompactionAdminClient.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/CompactionAdminClient.java
index 48042e7..a2ecb67 100644
--- a/hudi-client/src/main/java/org/apache/hudi/client/CompactionAdminClient.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/CompactionAdminClient.java
@@ -22,6 +22,7 @@
 import org.apache.hadoop.fs.Path;
 import org.apache.hudi.avro.model.HoodieCompactionOperation;
 import org.apache.hudi.avro.model.HoodieCompactionPlan;
+import org.apache.hudi.client.common.HoodieEngineContext;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.CompactionOperation;
 import org.apache.hudi.common.model.FileSlice;
@@ -45,7 +46,6 @@
 import org.apache.hudi.table.action.compact.OperationResult;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
-import org.apache.spark.api.java.JavaSparkContext;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -65,8 +65,8 @@
 
   private static final Logger LOG = LogManager.getLogger(CompactionAdminClient.class);
 
-  public CompactionAdminClient(JavaSparkContext jsc, String basePath) {
-    super(jsc, HoodieWriteConfig.newBuilder().withPath(basePath).build());
+  public CompactionAdminClient(HoodieEngineContext context, String basePath) {
+    super(context, HoodieWriteConfig.newBuilder().withPath(basePath).build());
   }
 
   /**
@@ -85,14 +85,14 @@
     if (plan.getOperations() != null) {
       List<CompactionOperation> ops = plan.getOperations().stream()
           .map(CompactionOperation::convertFromAvroRecordInstance).collect(Collectors.toList());
-      jsc.setJobGroup(this.getClass().getSimpleName(), "Validate compaction operations");
-      return jsc.parallelize(ops, parallelism).map(op -> {
+      context.setJobStatus(this.getClass().getSimpleName(), "Validate compaction operations");
+      return context.map(ops, op -> {
         try {
           return validateCompactionOperation(metaClient, compactionInstant, op, Option.of(fsView));
         } catch (IOException e) {
           throw new HoodieIOException(e.getMessage(), e);
         }
-      }).collect();
+      }, parallelism);
     }
     return new ArrayList<>();
   }
@@ -351,8 +351,8 @@
     } else {
       LOG.info("The following compaction renaming operations needs to be performed to un-schedule");
       if (!dryRun) {
-        jsc.setJobGroup(this.getClass().getSimpleName(), "Execute unschedule operations");
-        return jsc.parallelize(renameActions, parallelism).map(lfPair -> {
+        context.setJobStatus(this.getClass().getSimpleName(), "Execute unschedule operations");
+        return context.map(renameActions, lfPair -> {
           try {
             LOG.info("RENAME " + lfPair.getLeft().getPath() + " => " + lfPair.getRight().getPath());
             renameLogFile(metaClient, lfPair.getLeft(), lfPair.getRight());
@@ -363,7 +363,7 @@
                 + lfPair.getLeft().getBaseCommitTime() + "\" to recover from failure ***\n\n\n");
             return new RenameOpResult(lfPair, false, Option.of(e));
           }
-        }).collect();
+        }, parallelism);
       } else {
         LOG.info("Dry-Run Mode activated for rename operations");
         return renameActions.parallelStream().map(lfPair -> new RenameOpResult(lfPair, false, false, Option.empty()))
@@ -394,17 +394,17 @@
           "Number of Compaction Operations :" + plan.getOperations().size() + " for instant :" + compactionInstant);
       List<CompactionOperation> ops = plan.getOperations().stream()
           .map(CompactionOperation::convertFromAvroRecordInstance).collect(Collectors.toList());
-      jsc.setJobGroup(this.getClass().getSimpleName(), "Generate compaction unscheduling operations");
-      return jsc.parallelize(ops, parallelism).flatMap(op -> {
+      context.setJobStatus(this.getClass().getSimpleName(), "Generate compaction unscheduling operations");
+      return context.flatMap(ops, op -> {
         try {
           return getRenamingActionsForUnschedulingCompactionOperation(metaClient, compactionInstant, op,
-              Option.of(fsView), skipValidation).iterator();
+              Option.of(fsView), skipValidation).stream();
         } catch (IOException ioe) {
           throw new HoodieIOException(ioe.getMessage(), ioe);
         } catch (CompactionValidationException ve) {
           throw new HoodieException(ve);
         }
-      }).collect();
+      }, parallelism);
     }
     LOG.warn("No operations for compaction instant : " + compactionInstant);
     return new ArrayList<>();
diff --git a/hudi-client/src/main/java/org/apache/hudi/client/HoodieInternalWriteStatus.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieInternalWriteStatus.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/client/HoodieInternalWriteStatus.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieInternalWriteStatus.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/ReplaceArchivalHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/ReplaceArchivalHelper.java
similarity index 86%
rename from hudi-client/src/main/java/org/apache/hudi/table/ReplaceArchivalHelper.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/ReplaceArchivalHelper.java
index 317cf0d..515f43e 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/ReplaceArchivalHelper.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/ReplaceArchivalHelper.java
@@ -16,21 +16,23 @@
  * limitations under the License.
  */
 
-package org.apache.hudi.table;
+package org.apache.hudi.client;
 
 import com.fasterxml.jackson.databind.DeserializationFeature;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.hadoop.fs.Path;
+
 import org.apache.hudi.common.model.FileSlice;
+import org.apache.hudi.common.model.HoodieFileGroup;
 import org.apache.hudi.common.model.HoodieReplaceCommitMetadata;
 import org.apache.hudi.common.model.HoodieRollingStatMetadata;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
 import org.apache.hudi.common.table.timeline.HoodieInstant;
 import org.apache.hudi.common.table.view.TableFileSystemView;
+import org.apache.hudi.client.common.HoodieEngineContext;
+
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
-import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
 
 import java.io.IOException;
 import java.io.Serializable;
@@ -63,17 +65,17 @@
   /**
    * Delete all files represented by FileSlices in parallel. Return true if all files are deleted successfully.
    */
-  public static boolean deleteReplacedFileGroups(JavaSparkContext jsc, HoodieTableMetaClient metaClient,
+  public static boolean deleteReplacedFileGroups(HoodieEngineContext context, HoodieTableMetaClient metaClient,
                                                  TableFileSystemView fileSystemView,
                                                  HoodieInstant instant, List<String> replacedPartitions) {
 
-    JavaRDD<String> partitions = jsc.parallelize(replacedPartitions, replacedPartitions.size());
-    return partitions.map(partition -> {
+    List<Boolean> f = context.map(replacedPartitions, partition -> {
       Stream<FileSlice> fileSlices =  fileSystemView.getReplacedFileGroupsBeforeOrOn(instant.getTimestamp(), partition)
-          .flatMap(g -> g.getAllRawFileSlices());
+          .flatMap(HoodieFileGroup::getAllRawFileSlices);
+      return fileSlices.allMatch(slice -> deleteFileSlice(slice, metaClient, instant));
+    }, replacedPartitions.size());
 
-      return fileSlices.map(slice -> deleteFileSlice(slice, metaClient, instant)).allMatch(x -> x);
-    }).reduce((x, y) -> x & y);
+    return f.stream().reduce((x, y) -> x & y).orElse(true);
   }
 
   private static boolean deleteFileSlice(FileSlice fileSlice, HoodieTableMetaClient metaClient, HoodieInstant instant) {
@@ -95,5 +97,4 @@
       return false;
     }
   }
-
 }
diff --git a/hudi-client/src/main/java/org/apache/hudi/client/WriteStatus.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/WriteStatus.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/client/WriteStatus.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/WriteStatus.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/BootstrapMode.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/BootstrapMode.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/client/bootstrap/BootstrapMode.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/BootstrapMode.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/BootstrapRecordPayload.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/BootstrapRecordPayload.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/client/bootstrap/BootstrapRecordPayload.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/BootstrapRecordPayload.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/BootstrapWriteStatus.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/BootstrapWriteStatus.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/client/bootstrap/BootstrapWriteStatus.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/BootstrapWriteStatus.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/FullRecordBootstrapDataProvider.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/FullRecordBootstrapDataProvider.java
similarity index 75%
rename from hudi-client/src/main/java/org/apache/hudi/client/bootstrap/FullRecordBootstrapDataProvider.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/FullRecordBootstrapDataProvider.java
index 8b077ad..542dad9 100644
--- a/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/FullRecordBootstrapDataProvider.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/FullRecordBootstrapDataProvider.java
@@ -20,30 +20,28 @@
 
 import java.io.Serializable;
 import org.apache.hudi.avro.model.HoodieFileStatus;
+import org.apache.hudi.client.common.HoodieEngineContext;
 import org.apache.hudi.common.config.TypedProperties;
-import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.util.collection.Pair;
 
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
-import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
 
 import java.util.List;
 
 /**
- * Creates RDD of Hoodie Records with complete record data, given a list of partitions to be bootstrapped.
+ * Creates Hoodie Records with complete record data, given a list of partitions to be bootstrapped.
  */
-public abstract class FullRecordBootstrapDataProvider implements Serializable {
+public abstract class FullRecordBootstrapDataProvider<I> implements Serializable {
 
   protected static final Logger LOG = LogManager.getLogger(FullRecordBootstrapDataProvider.class);
 
   protected final TypedProperties props;
-  protected final transient JavaSparkContext jsc;
+  protected final transient HoodieEngineContext context;
 
-  public FullRecordBootstrapDataProvider(TypedProperties props, JavaSparkContext jsc) {
+  public FullRecordBootstrapDataProvider(TypedProperties props, HoodieEngineContext context) {
     this.props = props;
-    this.jsc = jsc;
+    this.context = context;
   }
 
   /**
@@ -51,8 +49,8 @@
    * @param tableName Hudi Table Name
    * @param sourceBasePath Source Base Path
    * @param partitionPaths Partition Paths
-   * @return JavaRDD of input records
+   * @return input records
    */
-  public abstract JavaRDD<HoodieRecord> generateInputRecordRDD(String tableName,
+  public abstract I generateInputRecords(String tableName,
       String sourceBasePath, List<Pair<String, List<HoodieFileStatus>>> partitionPaths);
 }
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/HoodieBootstrapSchemaProvider.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/HoodieBootstrapSchemaProvider.java
new file mode 100644
index 0000000..f63345d
--- /dev/null
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/HoodieBootstrapSchemaProvider.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.hudi.client.bootstrap;
+
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.avro.model.HoodieFileStatus;
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.config.HoodieWriteConfig;
+
+import org.apache.avro.Schema;
+
+import java.util.List;
+
+/**
+ * Bootstrap Schema Provider. Schema provided in config is used. If not available, use schema from Parquet
+ */
+public abstract class HoodieBootstrapSchemaProvider {
+
+  protected final HoodieWriteConfig writeConfig;
+
+  public HoodieBootstrapSchemaProvider(HoodieWriteConfig writeConfig) {
+    this.writeConfig = writeConfig;
+  }
+
+  /**
+   * Main API to select avro schema for bootstrapping.
+   * @param context HoodieEngineContext
+   * @param partitions  List of partitions with files within them
+   * @return Avro Schema
+   */
+  public final Schema getBootstrapSchema(HoodieEngineContext context, List<Pair<String, List<HoodieFileStatus>>> partitions) {
+    if (writeConfig.getSchema() != null) {
+      // Use schema specified by user if set
+      Schema userSchema = Schema.parse(writeConfig.getSchema());
+      if (!HoodieAvroUtils.getNullSchema().equals(userSchema)) {
+        return userSchema;
+      }
+    }
+    return getBootstrapSourceSchema(context, partitions);
+  }
+
+  /**
+   * Select a random file to be used to generate avro schema.
+   * Override this method to get custom schema selection.
+   * @param context HoodieEngineContext
+   * @param partitions  List of partitions with files within them
+   * @return Avro Schema
+   */
+  protected abstract Schema getBootstrapSourceSchema(HoodieEngineContext context,
+      List<Pair<String, List<HoodieFileStatus>>> partitions);
+
+}
diff --git a/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/selector/BootstrapModeSelector.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/selector/BootstrapModeSelector.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/client/bootstrap/selector/BootstrapModeSelector.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/selector/BootstrapModeSelector.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/selector/BootstrapRegexModeSelector.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/selector/BootstrapRegexModeSelector.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/client/bootstrap/selector/BootstrapRegexModeSelector.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/selector/BootstrapRegexModeSelector.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/selector/FullRecordBootstrapModeSelector.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/selector/FullRecordBootstrapModeSelector.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/client/bootstrap/selector/FullRecordBootstrapModeSelector.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/selector/FullRecordBootstrapModeSelector.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/selector/MetadataOnlyBootstrapModeSelector.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/selector/MetadataOnlyBootstrapModeSelector.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/client/bootstrap/selector/MetadataOnlyBootstrapModeSelector.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/selector/MetadataOnlyBootstrapModeSelector.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/selector/UniformBootstrapModeSelector.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/selector/UniformBootstrapModeSelector.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/client/bootstrap/selector/UniformBootstrapModeSelector.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/selector/UniformBootstrapModeSelector.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/translator/BootstrapPartitionPathTranslator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/translator/BootstrapPartitionPathTranslator.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/client/bootstrap/translator/BootstrapPartitionPathTranslator.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/translator/BootstrapPartitionPathTranslator.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/translator/IdentityBootstrapPartitionPathTranslator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/translator/IdentityBootstrapPartitionPathTranslator.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/client/bootstrap/translator/IdentityBootstrapPartitionPathTranslator.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/translator/IdentityBootstrapPartitionPathTranslator.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/keygen/KeyGeneratorInterface.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/EngineProperty.java
similarity index 61%
copy from hudi-client/src/main/java/org/apache/hudi/keygen/KeyGeneratorInterface.java
copy to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/EngineProperty.java
index 6412a2f..aeaec32 100644
--- a/hudi-client/src/main/java/org/apache/hudi/keygen/KeyGeneratorInterface.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/EngineProperty.java
@@ -16,26 +16,18 @@
  * limitations under the License.
  */
 
-package org.apache.hudi.keygen;
-
-import org.apache.avro.generic.GenericRecord;
-import org.apache.hudi.common.model.HoodieKey;
-import org.apache.spark.sql.Row;
-
-import java.io.Serializable;
-import java.util.List;
+package org.apache.hudi.client.common;
 
 /**
- * Represents the interface key generators need to adhere to.
+ * Properties specific to each engine, that can be set/obtained from.
  */
-public interface KeyGeneratorInterface extends Serializable {
-
-  HoodieKey getKey(GenericRecord record);
-
-  List<String> getRecordKeyFieldNames();
-
-  String getRecordKey(Row row);
-
-  String getPartitionPath(Row row);
-
+public enum EngineProperty {
+  // hostname to bind embedded timeline server to
+  EMBEDDED_SERVER_HOST,
+  // Pool/queue to use to run compaction.
+  COMPACTION_POOL_NAME,
+  // Amount of total memory available to each engine executor
+  TOTAL_MEMORY_AVAILABLE,
+  // Fraction of that memory, that is already in use by the engine
+  MEMORY_FRACTION_IN_USE,
 }
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/HoodieEngineContext.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/HoodieEngineContext.java
new file mode 100644
index 0000000..f8d8896
--- /dev/null
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/HoodieEngineContext.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.hudi.client.common;
+
+import org.apache.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.client.common.function.SerializableConsumer;
+import org.apache.hudi.client.common.function.SerializableFunction;
+import org.apache.hudi.client.common.function.SerializablePairFunction;
+import org.apache.hudi.common.util.Option;
+
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Stream;
+
+/**
+ * Base class contains the context information needed by the engine at runtime. It will be extended by different
+ * engine implementation if needed.
+ */
+public abstract class HoodieEngineContext {
+
+  /**
+   * A wrapped hadoop configuration which can be serialized.
+   */
+  private SerializableConfiguration hadoopConf;
+
+  private TaskContextSupplier taskContextSupplier;
+
+  public HoodieEngineContext(SerializableConfiguration hadoopConf, TaskContextSupplier taskContextSupplier) {
+    this.hadoopConf = hadoopConf;
+    this.taskContextSupplier = taskContextSupplier;
+  }
+
+  public SerializableConfiguration getHadoopConf() {
+    return hadoopConf;
+  }
+
+  public TaskContextSupplier getTaskContextSupplier() {
+    return taskContextSupplier;
+  }
+
+  public abstract <I, O> List<O> map(List<I> data, SerializableFunction<I, O> func, int parallelism);
+
+  public abstract <I, O> List<O> flatMap(List<I> data, SerializableFunction<I, Stream<O>> func, int parallelism);
+
+  public abstract <I> void foreach(List<I> data, SerializableConsumer<I> consumer, int parallelism);
+
+  public abstract <I, K, V> Map<K, V> mapToPair(List<I> data, SerializablePairFunction<I, K, V> func, Integer parallelism);
+
+  public abstract void setProperty(EngineProperty key, String value);
+
+  public abstract Option<String> getProperty(EngineProperty key);
+
+  public abstract void setJobStatus(String activeModule, String activityDescription);
+
+}
diff --git a/hudi-client/src/main/java/org/apache/hudi/keygen/KeyGeneratorInterface.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/TaskContextSupplier.java
similarity index 64%
copy from hudi-client/src/main/java/org/apache/hudi/keygen/KeyGeneratorInterface.java
copy to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/TaskContextSupplier.java
index 6412a2f..3a350d6 100644
--- a/hudi-client/src/main/java/org/apache/hudi/keygen/KeyGeneratorInterface.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/TaskContextSupplier.java
@@ -16,26 +16,23 @@
  * limitations under the License.
  */
 
-package org.apache.hudi.keygen;
+package org.apache.hudi.client.common;
 
-import org.apache.avro.generic.GenericRecord;
-import org.apache.hudi.common.model.HoodieKey;
-import org.apache.spark.sql.Row;
+import org.apache.hudi.common.util.Option;
 
 import java.io.Serializable;
-import java.util.List;
+import java.util.function.Supplier;
 
 /**
- * Represents the interface key generators need to adhere to.
+ * Base task context supplier.
  */
-public interface KeyGeneratorInterface extends Serializable {
+public abstract class TaskContextSupplier implements Serializable {
 
-  HoodieKey getKey(GenericRecord record);
+  public abstract Supplier<Integer> getPartitionIdSupplier();
 
-  List<String> getRecordKeyFieldNames();
+  public abstract Supplier<Integer> getStageIdSupplier();
 
-  String getRecordKey(Row row);
+  public abstract Supplier<Long> getAttemptIdSupplier();
 
-  String getPartitionPath(Row row);
-
+  public abstract Option<String> getProperty(EngineProperty prop);
 }
diff --git a/hudi-client/src/main/java/org/apache/hudi/keygen/KeyGeneratorInterface.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/function/SerializableConsumer.java
similarity index 64%
copy from hudi-client/src/main/java/org/apache/hudi/keygen/KeyGeneratorInterface.java
copy to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/function/SerializableConsumer.java
index 6412a2f..d7c4205 100644
--- a/hudi-client/src/main/java/org/apache/hudi/keygen/KeyGeneratorInterface.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/function/SerializableConsumer.java
@@ -16,26 +16,16 @@
  * limitations under the License.
  */
 
-package org.apache.hudi.keygen;
-
-import org.apache.avro.generic.GenericRecord;
-import org.apache.hudi.common.model.HoodieKey;
-import org.apache.spark.sql.Row;
+package org.apache.hudi.client.common.function;
 
 import java.io.Serializable;
-import java.util.List;
 
 /**
- * Represents the interface key generators need to adhere to.
+ * A wrapped {@link java.util.function.Consumer} which can be serialized.
+ *
+ * @param <I> input type
  */
-public interface KeyGeneratorInterface extends Serializable {
-
-  HoodieKey getKey(GenericRecord record);
-
-  List<String> getRecordKeyFieldNames();
-
-  String getRecordKey(Row row);
-
-  String getPartitionPath(Row row);
-
+@FunctionalInterface
+public interface SerializableConsumer<I> extends Serializable {
+  void accept(I t) throws Exception;
 }
diff --git a/hudi-client/src/main/java/org/apache/hudi/keygen/KeyGeneratorInterface.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/function/SerializableFunction.java
similarity index 64%
copy from hudi-client/src/main/java/org/apache/hudi/keygen/KeyGeneratorInterface.java
copy to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/function/SerializableFunction.java
index 6412a2f..d3714bc 100644
--- a/hudi-client/src/main/java/org/apache/hudi/keygen/KeyGeneratorInterface.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/function/SerializableFunction.java
@@ -16,26 +16,17 @@
  * limitations under the License.
  */
 
-package org.apache.hudi.keygen;
-
-import org.apache.avro.generic.GenericRecord;
-import org.apache.hudi.common.model.HoodieKey;
-import org.apache.spark.sql.Row;
+package org.apache.hudi.client.common.function;
 
 import java.io.Serializable;
-import java.util.List;
 
 /**
- * Represents the interface key generators need to adhere to.
+ * A wrapped {@link java.util.function.Function} which can be serialized.
+ *
+ * @param <I> input data type
+ * @param <O> output data type
  */
-public interface KeyGeneratorInterface extends Serializable {
-
-  HoodieKey getKey(GenericRecord record);
-
-  List<String> getRecordKeyFieldNames();
-
-  String getRecordKey(Row row);
-
-  String getPartitionPath(Row row);
-
+@FunctionalInterface
+public interface SerializableFunction<I, O> extends Serializable {
+  O apply(I v1) throws Exception;
 }
diff --git a/hudi-client/src/main/java/org/apache/hudi/keygen/KeyGeneratorInterface.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/function/SerializablePairFunction.java
similarity index 64%
copy from hudi-client/src/main/java/org/apache/hudi/keygen/KeyGeneratorInterface.java
copy to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/function/SerializablePairFunction.java
index 6412a2f..155837b 100644
--- a/hudi-client/src/main/java/org/apache/hudi/keygen/KeyGeneratorInterface.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/common/function/SerializablePairFunction.java
@@ -16,26 +16,16 @@
  * limitations under the License.
  */
 
-package org.apache.hudi.keygen;
+package org.apache.hudi.client.common.function;
 
-import org.apache.avro.generic.GenericRecord;
-import org.apache.hudi.common.model.HoodieKey;
-import org.apache.spark.sql.Row;
+import scala.Tuple2;
 
 import java.io.Serializable;
-import java.util.List;
 
 /**
- * Represents the interface key generators need to adhere to.
+ * A function that returns key-value pairs (Tuple2&lt;K, V&gt;).
  */
-public interface KeyGeneratorInterface extends Serializable {
-
-  HoodieKey getKey(GenericRecord record);
-
-  List<String> getRecordKeyFieldNames();
-
-  String getRecordKey(Row row);
-
-  String getPartitionPath(Row row);
-
+@FunctionalInterface
+public interface SerializablePairFunction<I, K, V> extends Serializable {
+  Tuple2<K, V> call(I t) throws Exception;
 }
diff --git a/hudi-client/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineService.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineService.java
similarity index 85%
rename from hudi-client/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineService.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineService.java
index 730585b..ca0c713 100644
--- a/hudi-client/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineService.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/embedded/EmbeddedTimelineService.java
@@ -18,6 +18,7 @@
 
 package org.apache.hudi.client.embedded;
 
+import org.apache.hudi.client.common.HoodieEngineContext;
 import org.apache.hudi.common.config.SerializableConfiguration;
 import org.apache.hudi.common.table.view.FileSystemViewManager;
 import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
@@ -25,10 +26,8 @@
 import org.apache.hudi.common.util.NetworkUtils;
 import org.apache.hudi.timeline.service.TimelineService;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
-import org.apache.spark.SparkConf;
 
 import java.io.IOException;
 
@@ -46,13 +45,10 @@
   private transient FileSystemViewManager viewManager;
   private transient TimelineService server;
 
-  public EmbeddedTimelineService(Configuration hadoopConf, SparkConf sparkConf, FileSystemViewStorageConfig config) {
-    setHostAddrFromSparkConf(sparkConf);
-    if (hostAddr == null) {
-      this.hostAddr = NetworkUtils.getHostname();
-    }
+  public EmbeddedTimelineService(HoodieEngineContext context, String embeddedTimelineServiceHostAddr, FileSystemViewStorageConfig config) {
+    setHostAddr(embeddedTimelineServiceHostAddr);
     this.config = config;
-    this.hadoopConf = new SerializableConfiguration(hadoopConf);
+    this.hadoopConf = context.getHadoopConf();
     this.viewManager = createViewManager();
   }
 
@@ -75,13 +71,13 @@
     LOG.info("Started embedded timeline server at " + hostAddr + ":" + serverPort);
   }
 
-  private void setHostAddrFromSparkConf(SparkConf sparkConf) {
-    String hostAddr = sparkConf.get("spark.driver.host", null);
-    if (hostAddr != null) {
-      LOG.info("Overriding hostIp to (" + hostAddr + ") found in spark-conf. It was " + this.hostAddr);
-      this.hostAddr = hostAddr;
+  private void setHostAddr(String embeddedTimelineServiceHostAddr) {
+    if (embeddedTimelineServiceHostAddr != null) {
+      LOG.info("Overriding hostIp to (" + embeddedTimelineServiceHostAddr + ") found in spark-conf. It was " + this.hostAddr);
+      this.hostAddr = embeddedTimelineServiceHostAddr;
     } else {
       LOG.warn("Unable to find driver bind address from spark config");
+      this.hostAddr = NetworkUtils.getHostname();
     }
   }
 
diff --git a/hudi-client/src/main/java/org/apache/hudi/client/utils/LazyIterableIterator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/LazyIterableIterator.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/client/utils/LazyIterableIterator.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/LazyIterableIterator.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/client/utils/MergingIterator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/MergingIterator.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/client/utils/MergingIterator.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/MergingIterator.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/config/HoodieBootstrapConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieBootstrapConfig.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/config/HoodieBootstrapConfig.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieBootstrapConfig.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/config/HoodieHBaseIndexConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieHBaseIndexConfig.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/config/HoodieHBaseIndexConfig.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieHBaseIndexConfig.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/config/HoodieIndexConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieIndexConfig.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/config/HoodieIndexConfig.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieIndexConfig.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/config/HoodieMemoryConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieMemoryConfig.java
similarity index 96%
rename from hudi-client/src/main/java/org/apache/hudi/config/HoodieMemoryConfig.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieMemoryConfig.java
index 3105fcd..687033f 100644
--- a/hudi-client/src/main/java/org/apache/hudi/config/HoodieMemoryConfig.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieMemoryConfig.java
@@ -118,6 +118,8 @@
           String.valueOf(DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE));
       setDefaultOnCondition(props, !props.containsKey(SPILLABLE_MAP_BASE_PATH_PROP), SPILLABLE_MAP_BASE_PATH_PROP,
           DEFAULT_SPILLABLE_MAP_BASE_PATH);
+      setDefaultOnCondition(props, !props.containsKey(MAX_MEMORY_FOR_MERGE_PROP), MAX_MEMORY_FOR_MERGE_PROP,
+          String.valueOf(DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES));
       setDefaultOnCondition(props, !props.containsKey(WRITESTATUS_FAILURE_FRACTION_PROP),
           WRITESTATUS_FAILURE_FRACTION_PROP, String.valueOf(DEFAULT_WRITESTATUS_FAILURE_FRACTION));
       return config;
diff --git a/hudi-client/src/main/java/org/apache/hudi/config/HoodieMetricsConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieMetricsConfig.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/config/HoodieMetricsConfig.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieMetricsConfig.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/config/HoodieMetricsDatadogConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieMetricsDatadogConfig.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/config/HoodieMetricsDatadogConfig.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieMetricsDatadogConfig.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/config/HoodieMetricsPrometheusConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieMetricsPrometheusConfig.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/config/HoodieMetricsPrometheusConfig.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieMetricsPrometheusConfig.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/config/HoodieStorageConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieStorageConfig.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/config/HoodieStorageConfig.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieStorageConfig.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/config/HoodieWriteCommitCallbackConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteCommitCallbackConfig.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/config/HoodieWriteCommitCallbackConfig.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteCommitCallbackConfig.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java
similarity index 99%
rename from hudi-client/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java
index 089474d..60ca3c2 100644
--- a/hudi-client/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java
@@ -19,7 +19,6 @@
 package org.apache.hudi.config;
 
 import org.apache.hadoop.hbase.io.compress.Compression;
-import org.apache.hudi.client.HoodieWriteClient;
 import org.apache.hudi.client.WriteStatus;
 import org.apache.hudi.client.bootstrap.BootstrapMode;
 import org.apache.hudi.common.config.DefaultHoodieConfig;
@@ -51,7 +50,7 @@
 import java.util.stream.Collectors;
 
 /**
- * Class storing configs for the {@link HoodieWriteClient}.
+ * Class storing configs for the HoodieWriteClient.
  */
 @Immutable
 public class HoodieWriteConfig extends DefaultHoodieConfig {
@@ -672,7 +671,7 @@
   public boolean getPushGatewayRandomJobNameSuffix() {
     return Boolean.parseBoolean(props.getProperty(HoodieMetricsPrometheusConfig.PUSHGATEWAY_RANDOM_JOB_NAME_SUFFIX));
   }
-  
+
   /**
    * memory configs.
    */
@@ -755,6 +754,10 @@
     return Integer.parseInt(props.getProperty(HoodieBootstrapConfig.BOOTSTRAP_PARALLELISM));
   }
 
+  public Long getMaxMemoryPerPartitionMerge() {
+    return Long.valueOf(props.getProperty(HoodieMemoryConfig.MAX_MEMORY_FOR_MERGE_PROP));
+  }
+
   public static class Builder {
 
     protected final Properties props = new Properties();
diff --git a/hudi-client/src/main/java/org/apache/hudi/exception/HoodieAppendException.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieAppendException.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/exception/HoodieAppendException.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieAppendException.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/exception/HoodieCommitCallbackException.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieCommitCallbackException.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/exception/HoodieCommitCallbackException.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieCommitCallbackException.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/exception/HoodieCommitException.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieCommitException.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/exception/HoodieCommitException.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieCommitException.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/exception/HoodieCompactionException.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieCompactionException.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/exception/HoodieCompactionException.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieCompactionException.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/exception/HoodieDependentSystemUnavailableException.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieDependentSystemUnavailableException.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/exception/HoodieDependentSystemUnavailableException.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieDependentSystemUnavailableException.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/exception/HoodieInsertException.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieInsertException.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/exception/HoodieInsertException.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieInsertException.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/exception/HoodieRestoreException.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieRestoreException.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/exception/HoodieRestoreException.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieRestoreException.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/exception/HoodieRollbackException.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieRollbackException.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/exception/HoodieRollbackException.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieRollbackException.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/exception/HoodieSavepointException.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieSavepointException.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/exception/HoodieSavepointException.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieSavepointException.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/exception/HoodieUpgradeDowngradeException.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieUpgradeDowngradeException.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/exception/HoodieUpgradeDowngradeException.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieUpgradeDowngradeException.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/exception/HoodieUpsertException.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieUpsertException.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/exception/HoodieUpsertException.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/exception/HoodieUpsertException.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/execution/CopyOnWriteInsertHandler.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/execution/CopyOnWriteInsertHandler.java
similarity index 84%
rename from hudi-client/src/main/java/org/apache/hudi/execution/CopyOnWriteInsertHandler.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/execution/CopyOnWriteInsertHandler.java
index ae61d80..8af72f3 100644
--- a/hudi-client/src/main/java/org/apache/hudi/execution/CopyOnWriteInsertHandler.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/execution/CopyOnWriteInsertHandler.java
@@ -18,13 +18,13 @@
 
 package org.apache.hudi.execution;
 
-import org.apache.hudi.client.SparkTaskContextSupplier;
 import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.TaskContextSupplier;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer;
 import org.apache.hudi.config.HoodieWriteConfig;
-import org.apache.hudi.execution.LazyInsertIterable.HoodieInsertValueGenResult;
+import org.apache.hudi.execution.HoodieLazyInsertIterable.HoodieInsertValueGenResult;
 import org.apache.hudi.io.HoodieWriteHandle;
 import org.apache.hudi.io.WriteHandleFactory;
 import org.apache.hudi.table.HoodieTable;
@@ -43,10 +43,10 @@
   private HoodieWriteConfig config;
   private String instantTime;
   private boolean areRecordsSorted;
-  private HoodieTable<T> hoodieTable;
+  private HoodieTable hoodieTable;
   private String idPrefix;
-  private SparkTaskContextSupplier sparkTaskContextSupplier;
-  private WriteHandleFactory<T> writeHandleFactory;
+  private TaskContextSupplier taskContextSupplier;
+  private WriteHandleFactory writeHandleFactory;
 
   private final List<WriteStatus> statuses = new ArrayList<>();
   // Stores the open HoodieWriteHandle for each table partition path
@@ -55,15 +55,15 @@
   private Map<String, HoodieWriteHandle> handles = new HashMap<>();
 
   public CopyOnWriteInsertHandler(HoodieWriteConfig config, String instantTime,
-                                  boolean areRecordsSorted, HoodieTable<T> hoodieTable, String idPrefix,
-                                  SparkTaskContextSupplier sparkTaskContextSupplier,
-                                  WriteHandleFactory<T> writeHandleFactory) {
+                                  boolean areRecordsSorted, HoodieTable hoodieTable, String idPrefix,
+                                  TaskContextSupplier taskContextSupplier,
+                                  WriteHandleFactory writeHandleFactory) {
     this.config = config;
     this.instantTime = instantTime;
     this.areRecordsSorted = areRecordsSorted;
     this.hoodieTable = hoodieTable;
     this.idPrefix = idPrefix;
-    this.sparkTaskContextSupplier = sparkTaskContextSupplier;
+    this.taskContextSupplier = taskContextSupplier;
     this.writeHandleFactory = writeHandleFactory;
   }
 
@@ -81,7 +81,7 @@
       }
       // Lazily initialize the handle, for the first time
       handle = writeHandleFactory.create(config, instantTime, hoodieTable,
-          insertPayload.getPartitionPath(), idPrefix, sparkTaskContextSupplier);
+          insertPayload.getPartitionPath(), idPrefix, taskContextSupplier);
       handles.put(partitionPath, handle);
     }
 
@@ -90,7 +90,7 @@
       statuses.add(handle.close());
       // Open new handle
       handle = writeHandleFactory.create(config, instantTime, hoodieTable,
-          insertPayload.getPartitionPath(), idPrefix, sparkTaskContextSupplier);
+          insertPayload.getPartitionPath(), idPrefix, taskContextSupplier);
       handles.put(partitionPath, handle);
     }
     handle.write(insertPayload, payload.insertValue, payload.exception);
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/execution/HoodieLazyInsertIterable.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/execution/HoodieLazyInsertIterable.java
new file mode 100644
index 0000000..b435c68
--- /dev/null
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/execution/HoodieLazyInsertIterable.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.hudi.execution;
+
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.utils.LazyIterableIterator;
+import org.apache.hudi.client.common.TaskContextSupplier;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.io.CreateHandleFactory;
+import org.apache.hudi.io.WriteHandleFactory;
+import org.apache.hudi.table.HoodieTable;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+
+import java.util.Iterator;
+import java.util.List;
+import java.util.function.Function;
+
+/**
+ * Lazy Iterable, that writes a stream of HoodieRecords sorted by the partitionPath, into new files.
+ */
+public abstract class HoodieLazyInsertIterable<T extends HoodieRecordPayload>
+    extends LazyIterableIterator<HoodieRecord<T>, List<WriteStatus>> {
+
+  protected final HoodieWriteConfig hoodieConfig;
+  protected final String instantTime;
+  protected boolean areRecordsSorted;
+  protected final HoodieTable hoodieTable;
+  protected final String idPrefix;
+  protected TaskContextSupplier taskContextSupplier;
+  protected WriteHandleFactory writeHandleFactory;
+
+  public HoodieLazyInsertIterable(Iterator<HoodieRecord<T>> recordItr,
+                                  boolean areRecordsSorted,
+                                  HoodieWriteConfig config,
+                                  String instantTime,
+                                  HoodieTable hoodieTable,
+                                  String idPrefix,
+                                  TaskContextSupplier taskContextSupplier) {
+    this(recordItr, areRecordsSorted, config, instantTime, hoodieTable, idPrefix, taskContextSupplier,
+        new CreateHandleFactory<>());
+  }
+
+  public HoodieLazyInsertIterable(Iterator<HoodieRecord<T>> recordItr, boolean areRecordsSorted,
+                                  HoodieWriteConfig config, String instantTime, HoodieTable hoodieTable,
+                                  String idPrefix, TaskContextSupplier taskContextSupplier,
+                                  WriteHandleFactory writeHandleFactory) {
+    super(recordItr);
+    this.areRecordsSorted = areRecordsSorted;
+    this.hoodieConfig = config;
+    this.instantTime = instantTime;
+    this.hoodieTable = hoodieTable;
+    this.idPrefix = idPrefix;
+    this.taskContextSupplier = taskContextSupplier;
+    this.writeHandleFactory = writeHandleFactory;
+  }
+
+  // Used for caching HoodieRecord along with insertValue. We need this to offload computation work to buffering thread.
+  public static class HoodieInsertValueGenResult<T extends HoodieRecord> {
+    public T record;
+    public Option<IndexedRecord> insertValue;
+    // It caches the exception seen while fetching insert value.
+    public Option<Exception> exception = Option.empty();
+
+    public HoodieInsertValueGenResult(T record, Schema schema) {
+      this.record = record;
+      try {
+        this.insertValue = record.getData().getInsertValue(schema);
+      } catch (Exception e) {
+        this.exception = Option.of(e);
+      }
+    }
+  }
+
+  /**
+   * Transformer function to help transform a HoodieRecord. This transformer is used by BufferedIterator to offload some
+   * expensive operations of transformation to the reader thread.
+   */
+  static <T extends HoodieRecordPayload> Function<HoodieRecord<T>, HoodieInsertValueGenResult<HoodieRecord>> getTransformFunction(
+      Schema schema) {
+    return hoodieRecord -> new HoodieInsertValueGenResult(hoodieRecord, schema);
+  }
+
+  @Override
+  protected void start() {}
+
+  @Override
+  protected void end() {}
+
+  protected CopyOnWriteInsertHandler getInsertHandler() {
+    return new CopyOnWriteInsertHandler(hoodieConfig, instantTime, areRecordsSorted, hoodieTable, idPrefix,
+        taskContextSupplier, writeHandleFactory);
+  }
+}
diff --git a/hudi-client/src/main/java/org/apache/hudi/execution/bulkinsert/BulkInsertSortMode.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/execution/bulkinsert/BulkInsertSortMode.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/execution/bulkinsert/BulkInsertSortMode.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/execution/bulkinsert/BulkInsertSortMode.java
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndex.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndex.java
new file mode 100644
index 0000000..c71b34e
--- /dev/null
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndex.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.hudi.index;
+
+import org.apache.hudi.ApiMaturityLevel;
+import org.apache.hudi.PublicAPIClass;
+import org.apache.hudi.PublicAPIMethod;
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.common.model.FileSlice;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieIndexException;
+import org.apache.hudi.table.HoodieTable;
+
+import java.io.Serializable;
+
+/**
+ * Base class for different types of indexes to determine the mapping from uuid.
+ *
+ * @param <T> Sub type of HoodieRecordPayload
+ * @param <I> Type of inputs
+ * @param <K> Type of keys
+ * @param <O> Type of outputs
+ */
+@PublicAPIClass(maturity = ApiMaturityLevel.EVOLVING)
+public abstract class HoodieIndex<T extends HoodieRecordPayload, I, K, O> implements Serializable {
+
+  protected final HoodieWriteConfig config;
+
+  protected HoodieIndex(HoodieWriteConfig config) {
+    this.config = config;
+  }
+
+  /**
+   * Looks up the index and tags each incoming record with a location of a file that contains the row (if it is actually
+   * present).
+   */
+  @PublicAPIMethod(maturity = ApiMaturityLevel.STABLE)
+  public abstract I tagLocation(I records, HoodieEngineContext context,
+                                HoodieTable<T, I, K, O> hoodieTable) throws HoodieIndexException;
+
+  /**
+   * Extracts the location of written records, and updates the index.
+   * <p>
+   * TODO(vc): We may need to propagate the record as well in a WriteStatus class
+   */
+  @PublicAPIMethod(maturity = ApiMaturityLevel.STABLE)
+  public abstract O updateLocation(O writeStatusRDD, HoodieEngineContext context,
+                                   HoodieTable<T, I, K, O> hoodieTable) throws HoodieIndexException;
+
+  /**
+   * Rollback the effects of the commit made at instantTime.
+   */
+  @PublicAPIMethod(maturity = ApiMaturityLevel.STABLE)
+  public abstract boolean rollbackCommit(String instantTime);
+
+  /**
+   * An index is `global` if {@link HoodieKey} to fileID mapping, does not depend on the `partitionPath`. Such an
+   * implementation is able to obtain the same mapping, for two hoodie keys with same `recordKey` but different
+   * `partitionPath`
+   *
+   * @return whether or not, the index implementation is global in nature
+   */
+  @PublicAPIMethod(maturity = ApiMaturityLevel.STABLE)
+  public abstract boolean isGlobal();
+
+  /**
+   * This is used by storage to determine, if its safe to send inserts, straight to the log, i.e having a
+   * {@link FileSlice}, with no data file.
+   *
+   * @return Returns true/false depending on whether the impl has this capability
+   */
+  @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING)
+  public abstract boolean canIndexLogFiles();
+
+  /**
+   * An index is "implicit" with respect to storage, if just writing new data to a file slice, updates the index as
+   * well. This is used by storage, to save memory footprint in certain cases.
+   */
+  @PublicAPIMethod(maturity = ApiMaturityLevel.STABLE)
+  public abstract boolean isImplicitWithStorage();
+
+  /**
+   * Each index type should implement it's own logic to release any resources acquired during the process.
+   */
+  public void close() {
+  }
+
+  public enum IndexType {
+    HBASE, INMEMORY, BLOOM, GLOBAL_BLOOM, SIMPLE, GLOBAL_SIMPLE
+  }
+}
diff --git a/hudi-client/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java
similarity index 75%
rename from hudi-client/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java
index ebd30b3..8cd0cb2 100644
--- a/hudi-client/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java
@@ -18,6 +18,7 @@
 
 package org.apache.hudi.index;
 
+import org.apache.hudi.client.common.HoodieEngineContext;
 import org.apache.hudi.common.model.HoodieBaseFile;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieRecordLocation;
@@ -26,8 +27,6 @@
 import org.apache.hudi.common.util.collection.Pair;
 import org.apache.hudi.table.HoodieTable;
 
-import org.apache.spark.api.java.JavaSparkContext;
-
 import java.util.ArrayList;
 import java.util.List;
 
@@ -42,28 +41,26 @@
    * Fetches Pair of partition path and {@link HoodieBaseFile}s for interested partitions.
    *
    * @param partitions  list of partitions of interest
-   * @param jsc         instance of {@link JavaSparkContext} to use
+   * @param context     instance of {@link HoodieEngineContext} to use
    * @param hoodieTable instance of {@link HoodieTable} of interest
    * @return the list of Pairs of partition path and fileId
    */
   public static List<Pair<String, HoodieBaseFile>> getLatestBaseFilesForAllPartitions(final List<String> partitions,
-                                                                                      final JavaSparkContext jsc,
+                                                                                      final HoodieEngineContext context,
                                                                                       final HoodieTable hoodieTable) {
-    jsc.setJobGroup(HoodieIndexUtils.class.getSimpleName(), "Load latest base files from all partitions");
-    return jsc.parallelize(partitions, Math.max(partitions.size(), 1))
-        .flatMap(partitionPath -> {
-          Option<HoodieInstant> latestCommitTime = hoodieTable.getMetaClient().getCommitsTimeline()
-              .filterCompletedInstants().lastInstant();
-          List<Pair<String, HoodieBaseFile>> filteredFiles = new ArrayList<>();
-          if (latestCommitTime.isPresent()) {
-            filteredFiles = hoodieTable.getBaseFileOnlyView()
-                .getLatestBaseFilesBeforeOrOn(partitionPath, latestCommitTime.get().getTimestamp())
-                .map(f -> Pair.of(partitionPath, f))
-                .collect(toList());
-          }
-          return filteredFiles.iterator();
-        })
-        .collect();
+    context.setJobStatus(HoodieIndexUtils.class.getSimpleName(), "Load latest base files from all partitions");
+    return context.flatMap(partitions, partitionPath -> {
+      Option<HoodieInstant> latestCommitTime = hoodieTable.getMetaClient().getCommitsTimeline()
+          .filterCompletedInstants().lastInstant();
+      List<Pair<String, HoodieBaseFile>> filteredFiles = new ArrayList<>();
+      if (latestCommitTime.isPresent()) {
+        filteredFiles = hoodieTable.getBaseFileOnlyView()
+            .getLatestBaseFilesBeforeOrOn(partitionPath, latestCommitTime.get().getTimestamp())
+            .map(f -> Pair.of(partitionPath, f))
+            .collect(toList());
+      }
+      return filteredFiles.stream();
+    }, Math.max(partitions.size(), 1));
   }
 
   /**
diff --git a/hudi-client/src/main/java/org/apache/hudi/index/bloom/BloomIndexFileInfo.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/BloomIndexFileInfo.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/index/bloom/BloomIndexFileInfo.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/BloomIndexFileInfo.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/index/bloom/IndexFileFilter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/IndexFileFilter.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/index/bloom/IndexFileFilter.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/IndexFileFilter.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/index/bloom/IntervalTreeBasedGlobalIndexFileFilter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/IntervalTreeBasedGlobalIndexFileFilter.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/index/bloom/IntervalTreeBasedGlobalIndexFileFilter.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/IntervalTreeBasedGlobalIndexFileFilter.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/index/bloom/IntervalTreeBasedIndexFileFilter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/IntervalTreeBasedIndexFileFilter.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/index/bloom/IntervalTreeBasedIndexFileFilter.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/IntervalTreeBasedIndexFileFilter.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/index/bloom/KeyRangeLookupTree.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/KeyRangeLookupTree.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/index/bloom/KeyRangeLookupTree.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/KeyRangeLookupTree.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/index/bloom/KeyRangeNode.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/KeyRangeNode.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/index/bloom/KeyRangeNode.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/KeyRangeNode.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/index/bloom/ListBasedGlobalIndexFileFilter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/ListBasedGlobalIndexFileFilter.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/index/bloom/ListBasedGlobalIndexFileFilter.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/ListBasedGlobalIndexFileFilter.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/index/bloom/ListBasedIndexFileFilter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/ListBasedIndexFileFilter.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/index/bloom/ListBasedIndexFileFilter.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/ListBasedIndexFileFilter.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/index/hbase/DefaultHBaseQPSResourceAllocator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/hbase/DefaultHBaseQPSResourceAllocator.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/index/hbase/DefaultHBaseQPSResourceAllocator.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/hbase/DefaultHBaseQPSResourceAllocator.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/index/hbase/HBaseIndexQPSResourceAllocator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/hbase/HBaseIndexQPSResourceAllocator.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/index/hbase/HBaseIndexQPSResourceAllocator.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/hbase/HBaseIndexQPSResourceAllocator.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/io/AppendHandleFactory.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/AppendHandleFactory.java
similarity index 73%
rename from hudi-client/src/main/java/org/apache/hudi/io/AppendHandleFactory.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/AppendHandleFactory.java
index 4a5554b..5c54dce 100644
--- a/hudi-client/src/main/java/org/apache/hudi/io/AppendHandleFactory.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/AppendHandleFactory.java
@@ -18,17 +18,17 @@
 
 package org.apache.hudi.io;
 
-import org.apache.hudi.client.SparkTaskContextSupplier;
+import org.apache.hudi.client.common.TaskContextSupplier;
 import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.table.HoodieTable;
 
-public class AppendHandleFactory<T extends HoodieRecordPayload> extends WriteHandleFactory<T> {
+public class AppendHandleFactory<T extends HoodieRecordPayload, I, K, O> extends WriteHandleFactory<T, I, K, O> {
 
   @Override
-  public HoodieAppendHandle<T> create(final HoodieWriteConfig hoodieConfig, final String commitTime,
-                                     final HoodieTable<T> hoodieTable, final String partitionPath,
-                                     final String fileIdPrefix, final SparkTaskContextSupplier sparkTaskContextSupplier) {
+  public HoodieAppendHandle<T, I, K, O> create(final HoodieWriteConfig hoodieConfig, final String commitTime,
+                                     final HoodieTable<T, I, K, O> hoodieTable, final String partitionPath,
+                                     final String fileIdPrefix, final TaskContextSupplier sparkTaskContextSupplier) {
 
     return new HoodieAppendHandle(hoodieConfig, commitTime, hoodieTable, partitionPath,
         getNextFileId(fileIdPrefix), sparkTaskContextSupplier);
diff --git a/hudi-client/src/main/java/org/apache/hudi/io/CreateHandleFactory.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/CreateHandleFactory.java
similarity index 70%
rename from hudi-client/src/main/java/org/apache/hudi/io/CreateHandleFactory.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/CreateHandleFactory.java
index 68d8b4d..67ebadb 100644
--- a/hudi-client/src/main/java/org/apache/hudi/io/CreateHandleFactory.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/CreateHandleFactory.java
@@ -18,19 +18,19 @@
 
 package org.apache.hudi.io;
 
-import org.apache.hudi.client.SparkTaskContextSupplier;
+import org.apache.hudi.client.common.TaskContextSupplier;
 import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.table.HoodieTable;
 
-public class CreateHandleFactory<T extends HoodieRecordPayload> extends WriteHandleFactory<T> {
+public class CreateHandleFactory<T extends HoodieRecordPayload, I, K, O> extends WriteHandleFactory<T, I, K, O> {
 
   @Override
-  public HoodieWriteHandle<T> create(final HoodieWriteConfig hoodieConfig, final String commitTime,
-                                     final HoodieTable<T> hoodieTable, final String partitionPath,
-                                     final String fileIdPrefix, SparkTaskContextSupplier sparkTaskContextSupplier) {
+  public HoodieWriteHandle<T, I, K, O> create(final HoodieWriteConfig hoodieConfig, final String commitTime,
+                                     final HoodieTable<T, I, K, O> hoodieTable, final String partitionPath,
+                                     final String fileIdPrefix, TaskContextSupplier taskContextSupplier) {
 
     return new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, partitionPath,
-        getNextFileId(fileIdPrefix), sparkTaskContextSupplier);
+        getNextFileId(fileIdPrefix), taskContextSupplier);
   }
 }
\ No newline at end of file
diff --git a/hudi-client/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java
similarity index 94%
rename from hudi-client/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java
index 7996a77..0c590fe 100644
--- a/hudi-client/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java
@@ -19,8 +19,8 @@
 package org.apache.hudi.io;
 
 import org.apache.hudi.avro.HoodieAvroUtils;
-import org.apache.hudi.client.SparkTaskContextSupplier;
 import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.TaskContextSupplier;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.FileSlice;
 import org.apache.hudi.common.model.HoodieDeltaWriteStat;
@@ -40,7 +40,9 @@
 import org.apache.hudi.common.table.log.block.HoodieLogBlock;
 import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType;
 import org.apache.hudi.common.table.view.TableFileSystemView.SliceView;
+import org.apache.hudi.common.util.DefaultSizeEstimator;
 import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.SizeEstimator;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.exception.HoodieAppendException;
 import org.apache.hudi.exception.HoodieUpsertException;
@@ -51,7 +53,6 @@
 import org.apache.hadoop.fs.Path;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
-import org.apache.spark.util.SizeEstimator;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -64,7 +65,7 @@
 /**
  * IO Operation to append data onto an existing file.
  */
-public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieWriteHandle<T> {
+public class HoodieAppendHandle<T extends HoodieRecordPayload, I, K, O> extends HoodieWriteHandle<T, I, K, O> {
 
   private static final Logger LOG = LogManager.getLogger(HoodieAppendHandle.class);
   // This acts as the sequenceID for records written
@@ -101,16 +102,19 @@
   // Total number of new records inserted into the delta file
   private long insertRecordsWritten = 0;
 
-  public HoodieAppendHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
-                            String partitionPath, String fileId, Iterator<HoodieRecord<T>> recordItr, SparkTaskContextSupplier sparkTaskContextSupplier) {
-    super(config, instantTime, partitionPath, fileId, hoodieTable, sparkTaskContextSupplier);
+  private SizeEstimator<HoodieRecord> sizeEstimator;
+
+  public HoodieAppendHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T, I, K, O> hoodieTable,
+                            String partitionPath, String fileId, Iterator<HoodieRecord<T>> recordItr, TaskContextSupplier taskContextSupplier) {
+    super(config, instantTime, partitionPath, fileId, hoodieTable, taskContextSupplier);
     writeStatus.setStat(new HoodieDeltaWriteStat());
     this.fileId = fileId;
     this.recordItr = recordItr;
+    sizeEstimator = new DefaultSizeEstimator();
   }
 
-  public HoodieAppendHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
-                            String partitionPath, String fileId, SparkTaskContextSupplier sparkTaskContextSupplier) {
+  public HoodieAppendHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T, I, K, O> hoodieTable,
+                            String partitionPath, String fileId, TaskContextSupplier sparkTaskContextSupplier) {
     this(config, instantTime, hoodieTable, partitionPath, fileId, null, sparkTaskContextSupplier);
   }
 
@@ -134,7 +138,7 @@
       writeStatus.setPartitionPath(partitionPath);
       writeStatus.getStat().setPartitionPath(partitionPath);
       writeStatus.getStat().setFileId(fileId);
-      averageRecordSize = SizeEstimator.estimate(record);
+      averageRecordSize = sizeEstimator.sizeEstimate(record);
       try {
         //save hoodie partition meta in the partition path
         HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs, baseInstantTime,
@@ -335,7 +339,7 @@
       // Recompute averageRecordSize before writing a new block and update existing value with
       // avg of new and old
       LOG.info("AvgRecordSize => " + averageRecordSize);
-      averageRecordSize = (averageRecordSize + SizeEstimator.estimate(record)) / 2;
+      averageRecordSize = (averageRecordSize + sizeEstimator.sizeEstimate(record)) / 2;
       doAppend(header);
       estimatedNumberOfBytesWritten += averageRecordSize * numberOfRecords;
       numberOfRecords = 0;
diff --git a/hudi-client/src/main/java/org/apache/hudi/io/HoodieBootstrapHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieBootstrapHandle.java
similarity index 87%
rename from hudi-client/src/main/java/org/apache/hudi/io/HoodieBootstrapHandle.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieBootstrapHandle.java
index 5deeae1..b2eaedd 100644
--- a/hudi-client/src/main/java/org/apache/hudi/io/HoodieBootstrapHandle.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieBootstrapHandle.java
@@ -19,7 +19,7 @@
 package org.apache.hudi.io;
 
 import org.apache.hudi.avro.HoodieAvroUtils;
-import org.apache.hudi.client.SparkTaskContextSupplier;
+import org.apache.hudi.client.common.TaskContextSupplier;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.util.collection.Pair;
@@ -33,13 +33,13 @@
  *    writing more than 1 skeleton file for the same bootstrap file.
  * @param <T> HoodieRecordPayload
  */
-public class HoodieBootstrapHandle<T extends HoodieRecordPayload> extends HoodieCreateHandle<T> {
+public class HoodieBootstrapHandle<T extends HoodieRecordPayload, I, K, O> extends HoodieCreateHandle<T, I, K, O> {
 
-  public HoodieBootstrapHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable,
-      String partitionPath, String fileId, SparkTaskContextSupplier sparkTaskContextSupplier) {
+  public HoodieBootstrapHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T, I, K, O> hoodieTable,
+      String partitionPath, String fileId, TaskContextSupplier taskContextSupplier) {
     super(config, commitTime, hoodieTable, partitionPath, fileId,
         Pair.of(HoodieAvroUtils.RECORD_KEY_SCHEMA,
-            HoodieAvroUtils.addMetadataFields(HoodieAvroUtils.RECORD_KEY_SCHEMA)), sparkTaskContextSupplier);
+            HoodieAvroUtils.addMetadataFields(HoodieAvroUtils.RECORD_KEY_SCHEMA)), taskContextSupplier);
   }
 
   @Override
diff --git a/hudi-client/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java
similarity index 92%
rename from hudi-client/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java
index 9953278..6a8e773 100644
--- a/hudi-client/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java
@@ -19,8 +19,8 @@
 package org.apache.hudi.io;
 
 import org.apache.avro.Schema;
-import org.apache.hudi.client.SparkTaskContextSupplier;
 import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.TaskContextSupplier;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.HoodiePartitionMetadata;
 import org.apache.hudi.common.model.HoodieRecord;
@@ -47,7 +47,7 @@
 import java.util.Iterator;
 import java.util.Map;
 
-public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieWriteHandle<T> {
+public class HoodieCreateHandle<T extends HoodieRecordPayload, I, K, O> extends HoodieWriteHandle<T, I, K, O> {
 
   private static final Logger LOG = LogManager.getLogger(HoodieCreateHandle.class);
 
@@ -59,17 +59,17 @@
   private Map<String, HoodieRecord<T>> recordMap;
   private boolean useWriterSchema = false;
 
-  public HoodieCreateHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
-                            String partitionPath, String fileId, SparkTaskContextSupplier sparkTaskContextSupplier) {
+  public HoodieCreateHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T, I, K, O> hoodieTable,
+                            String partitionPath, String fileId, TaskContextSupplier taskContextSupplier) {
     this(config, instantTime, hoodieTable, partitionPath, fileId, getWriterSchemaIncludingAndExcludingMetadataPair(config),
-        sparkTaskContextSupplier);
+        taskContextSupplier);
   }
 
-  public HoodieCreateHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
+  public HoodieCreateHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T, I, K, O> hoodieTable,
                             String partitionPath, String fileId, Pair<Schema, Schema> writerSchemaIncludingAndExcludingMetadataPair,
-                            SparkTaskContextSupplier sparkTaskContextSupplier) {
+                            TaskContextSupplier taskContextSupplier) {
     super(config, instantTime, partitionPath, fileId, hoodieTable, writerSchemaIncludingAndExcludingMetadataPair,
-            sparkTaskContextSupplier);
+        taskContextSupplier);
     writeStatus.setFileId(fileId);
     writeStatus.setPartitionPath(partitionPath);
 
@@ -80,7 +80,7 @@
           new Path(config.getBasePath()), FSUtils.getPartitionPath(config.getBasePath(), partitionPath));
       partitionMetadata.trySave(getPartitionId());
       createMarkerFile(partitionPath, FSUtils.makeDataFileName(this.instantTime, this.writeToken, this.fileId, hoodieTable.getBaseFileExtension()));
-      this.fileWriter = HoodieFileWriterFactory.getFileWriter(instantTime, path, hoodieTable, config, writerSchemaWithMetafields, this.sparkTaskContextSupplier);
+      this.fileWriter = HoodieFileWriterFactory.getFileWriter(instantTime, path, hoodieTable, config, writerSchemaWithMetafields, this.taskContextSupplier);
     } catch (IOException e) {
       throw new HoodieInsertException("Failed to initialize HoodieStorageWriter for path " + path, e);
     }
@@ -90,10 +90,10 @@
   /**
    * Called by the compactor code path.
    */
-  public HoodieCreateHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
+  public HoodieCreateHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T, I, K, O> hoodieTable,
       String partitionPath, String fileId, Map<String, HoodieRecord<T>> recordMap,
-      SparkTaskContextSupplier sparkTaskContextSupplier) {
-    this(config, instantTime, hoodieTable, partitionPath, fileId, sparkTaskContextSupplier);
+      TaskContextSupplier taskContextSupplier) {
+    this(config, instantTime, hoodieTable, partitionPath, fileId, taskContextSupplier);
     this.recordMap = recordMap;
     this.useWriterSchema = true;
   }
diff --git a/hudi-client/src/main/java/org/apache/hudi/io/HoodieIOHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieIOHandle.java
similarity index 92%
rename from hudi-client/src/main/java/org/apache/hudi/io/HoodieIOHandle.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieIOHandle.java
index cff91f3..c6f9ddd 100644
--- a/hudi-client/src/main/java/org/apache/hudi/io/HoodieIOHandle.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieIOHandle.java
@@ -24,14 +24,14 @@
 
 import org.apache.hadoop.fs.FileSystem;
 
-public abstract class HoodieIOHandle<T extends HoodieRecordPayload> {
+public abstract class HoodieIOHandle<T extends HoodieRecordPayload, I, K, O> {
 
   protected final String instantTime;
   protected final HoodieWriteConfig config;
   protected final FileSystem fs;
-  protected final HoodieTable<T> hoodieTable;
+  protected final HoodieTable<T, I, K, O> hoodieTable;
 
-  HoodieIOHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable) {
+  HoodieIOHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T, I, K, O> hoodieTable) {
     this.instantTime = instantTime;
     this.config = config;
     this.hoodieTable = hoodieTable;
diff --git a/hudi-client/src/main/java/org/apache/hudi/io/HoodieKeyLocationFetchHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLocationFetchHandle.java
similarity index 95%
rename from hudi-client/src/main/java/org/apache/hudi/io/HoodieKeyLocationFetchHandle.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLocationFetchHandle.java
index 3aa1398..9194fc0 100644
--- a/hudi-client/src/main/java/org/apache/hudi/io/HoodieKeyLocationFetchHandle.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLocationFetchHandle.java
@@ -38,11 +38,11 @@
  *
  * @param <T>
  */
-public class HoodieKeyLocationFetchHandle<T extends HoodieRecordPayload> extends HoodieReadHandle<T> {
+public class HoodieKeyLocationFetchHandle<T extends HoodieRecordPayload, I, K, O> extends HoodieReadHandle<T, I, K, O> {
 
   private final Pair<String, HoodieBaseFile> partitionPathBaseFilePair;
 
-  public HoodieKeyLocationFetchHandle(HoodieWriteConfig config, HoodieTable<T> hoodieTable,
+  public HoodieKeyLocationFetchHandle(HoodieWriteConfig config, HoodieTable<T, I, K, O> hoodieTable,
                                       Pair<String, HoodieBaseFile> partitionPathBaseFilePair) {
     super(config, null, hoodieTable, Pair.of(partitionPathBaseFilePair.getLeft(), partitionPathBaseFilePair.getRight().getFileId()));
     this.partitionPathBaseFilePair = partitionPathBaseFilePair;
diff --git a/hudi-client/src/main/java/org/apache/hudi/io/HoodieKeyLookupHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLookupHandle.java
similarity index 98%
rename from hudi-client/src/main/java/org/apache/hudi/io/HoodieKeyLookupHandle.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLookupHandle.java
index 4002b5a..ad84e3e 100644
--- a/hudi-client/src/main/java/org/apache/hudi/io/HoodieKeyLookupHandle.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLookupHandle.java
@@ -42,7 +42,7 @@
 /**
  * Takes a bunch of keys and returns ones that are present in the file group.
  */
-public class HoodieKeyLookupHandle<T extends HoodieRecordPayload> extends HoodieReadHandle<T> {
+public class HoodieKeyLookupHandle<T extends HoodieRecordPayload, I, K, O> extends HoodieReadHandle<T, I, K, O> {
 
   private static final Logger LOG = LogManager.getLogger(HoodieKeyLookupHandle.class);
 
@@ -54,7 +54,7 @@
 
   private long totalKeysChecked;
 
-  public HoodieKeyLookupHandle(HoodieWriteConfig config, HoodieTable<T> hoodieTable,
+  public HoodieKeyLookupHandle(HoodieWriteConfig config, HoodieTable<T, I, K, O> hoodieTable,
                                Pair<String, String> partitionPathFilePair) {
     super(config, null, hoodieTable, partitionPathFilePair);
     this.tableType = hoodieTable.getMetaClient().getTableType();
diff --git a/hudi-client/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java
similarity index 93%
rename from hudi-client/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java
index 650237c..77fef5c 100644
--- a/hudi-client/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java
@@ -18,9 +18,8 @@
 
 package org.apache.hudi.io;
 
-import org.apache.hudi.client.SparkTaskContextSupplier;
 import org.apache.hudi.client.WriteStatus;
-import org.apache.hudi.client.utils.SparkConfigUtils;
+import org.apache.hudi.client.common.TaskContextSupplier;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.HoodieBaseFile;
 import org.apache.hudi.common.model.HoodiePartitionMetadata;
@@ -54,7 +53,7 @@
 import java.util.Set;
 
 @SuppressWarnings("Duplicates")
-public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieWriteHandle<T> {
+public class HoodieMergeHandle<T extends HoodieRecordPayload, I, K, O> extends HoodieWriteHandle<T, I, K, O> {
 
   private static final Logger LOG = LogManager.getLogger(HoodieMergeHandle.class);
 
@@ -71,9 +70,10 @@
   protected boolean useWriterSchema;
   private HoodieBaseFile baseFileToMerge;
 
-  public HoodieMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
-       Iterator<HoodieRecord<T>> recordItr, String partitionPath, String fileId, SparkTaskContextSupplier sparkTaskContextSupplier) {
-    super(config, instantTime, partitionPath, fileId, hoodieTable, sparkTaskContextSupplier);
+  public HoodieMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T, I, K, O> hoodieTable,
+                           Iterator<HoodieRecord<T>> recordItr, String partitionPath, String fileId,
+                           TaskContextSupplier taskContextSupplier) {
+    super(config, instantTime, partitionPath, fileId, hoodieTable, taskContextSupplier);
     init(fileId, recordItr);
     init(fileId, partitionPath, hoodieTable.getBaseFileOnlyView().getLatestBaseFile(partitionPath, fileId).get());
   }
@@ -81,10 +81,10 @@
   /**
    * Called by compactor code path.
    */
-  public HoodieMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
-      Map<String, HoodieRecord<T>> keyToNewRecords, String partitionPath, String fileId,
-      HoodieBaseFile dataFileToBeMerged, SparkTaskContextSupplier sparkTaskContextSupplier) {
-    super(config, instantTime, partitionPath, fileId, hoodieTable, sparkTaskContextSupplier);
+  public HoodieMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T, I, K, O> hoodieTable,
+                           Map<String, HoodieRecord<T>> keyToNewRecords, String partitionPath, String fileId,
+                           HoodieBaseFile dataFileToBeMerged, TaskContextSupplier taskContextSupplier) {
+    super(config, instantTime, partitionPath, fileId, hoodieTable, taskContextSupplier);
     this.keyToNewRecords = keyToNewRecords;
     this.useWriterSchema = true;
     init(fileId, this.partitionPath, dataFileToBeMerged);
@@ -134,7 +134,7 @@
       createMarkerFile(partitionPath, newFileName);
 
       // Create the writer for writing the new version file
-      fileWriter = createNewFileWriter(instantTime, newFilePath, hoodieTable, config, writerSchemaWithMetafields, sparkTaskContextSupplier);
+      fileWriter = createNewFileWriter(instantTime, newFilePath, hoodieTable, config, writerSchemaWithMetafields, taskContextSupplier);
     } catch (IOException io) {
       LOG.error("Error in update task at commit " + instantTime, io);
       writeStatus.setGlobalError(io);
@@ -149,7 +149,7 @@
   private void init(String fileId, Iterator<HoodieRecord<T>> newRecordsItr) {
     try {
       // Load the new records in a map
-      long memoryForMerge = SparkConfigUtils.getMaxMemoryPerPartitionMerge(config.getProps());
+      long memoryForMerge = IOUtils.getMaxMemoryPerPartitionMerge(taskContextSupplier, config.getProps());
       LOG.info("MaxMemoryPerPartitionMerge => " + memoryForMerge);
       this.keyToNewRecords = new ExternalSpillableMap<>(memoryForMerge, config.getSpillableMapBasePath(),
           new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(writerSchema));
diff --git a/hudi-client/src/main/java/org/apache/hudi/io/HoodieRangeInfoHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieRangeInfoHandle.java
similarity index 94%
rename from hudi-client/src/main/java/org/apache/hudi/io/HoodieRangeInfoHandle.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieRangeInfoHandle.java
index 2b58583..78fa9be 100644
--- a/hudi-client/src/main/java/org/apache/hudi/io/HoodieRangeInfoHandle.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieRangeInfoHandle.java
@@ -28,9 +28,9 @@
 /**
  * Extract range information for a given file slice.
  */
-public class HoodieRangeInfoHandle<T extends HoodieRecordPayload> extends HoodieReadHandle<T> {
+public class HoodieRangeInfoHandle<T extends HoodieRecordPayload, I, K, O> extends HoodieReadHandle<T, I, K, O> {
 
-  public HoodieRangeInfoHandle(HoodieWriteConfig config, HoodieTable<T> hoodieTable,
+  public HoodieRangeInfoHandle(HoodieWriteConfig config, HoodieTable<T, I, K, O> hoodieTable,
       Pair<String, String> partitionPathFilePair) {
     super(config, null, hoodieTable, partitionPathFilePair);
   }
diff --git a/hudi-client/src/main/java/org/apache/hudi/io/HoodieReadHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieReadHandle.java
similarity index 95%
rename from hudi-client/src/main/java/org/apache/hudi/io/HoodieReadHandle.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieReadHandle.java
index 8f2d4de..a771c33 100644
--- a/hudi-client/src/main/java/org/apache/hudi/io/HoodieReadHandle.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieReadHandle.java
@@ -34,11 +34,11 @@
 /**
  * Base class for read operations done logically on the file group.
  */
-public abstract class HoodieReadHandle<T extends HoodieRecordPayload> extends HoodieIOHandle {
+public abstract class HoodieReadHandle<T extends HoodieRecordPayload, I, K, O> extends HoodieIOHandle<T, I, K, O> {
 
   protected final Pair<String, String> partitionPathFilePair;
 
-  public HoodieReadHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
+  public HoodieReadHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T, I, K, O> hoodieTable,
       Pair<String, String> partitionPathFilePair) {
     super(config, instantTime, hoodieTable);
     this.partitionPathFilePair = partitionPathFilePair;
diff --git a/hudi-client/src/main/java/org/apache/hudi/io/HoodieSortedMergeHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieSortedMergeHandle.java
similarity index 91%
rename from hudi-client/src/main/java/org/apache/hudi/io/HoodieSortedMergeHandle.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieSortedMergeHandle.java
index dda7b72..71610b1 100644
--- a/hudi-client/src/main/java/org/apache/hudi/io/HoodieSortedMergeHandle.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieSortedMergeHandle.java
@@ -18,8 +18,8 @@
 
 package org.apache.hudi.io;
 
-import org.apache.hudi.client.SparkTaskContextSupplier;
 import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.TaskContextSupplier;
 import org.apache.hudi.common.model.HoodieBaseFile;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieRecordPayload;
@@ -41,24 +41,24 @@
  * The implementation performs a merge-sort by comparing the key of the record being written to the list of
  * keys in newRecordKeys (sorted in-memory).
  */
-public class HoodieSortedMergeHandle<T extends HoodieRecordPayload> extends HoodieMergeHandle<T> {
+public class HoodieSortedMergeHandle<T extends HoodieRecordPayload, I, K, O> extends HoodieMergeHandle<T, I, K, O> {
 
   private Queue<String> newRecordKeysSorted = new PriorityQueue<>();
 
-  public HoodieSortedMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
-       Iterator<HoodieRecord<T>> recordItr, String partitionPath, String fileId, SparkTaskContextSupplier sparkTaskContextSupplier) {
-    super(config, instantTime, hoodieTable, recordItr, partitionPath, fileId, sparkTaskContextSupplier);
+  public HoodieSortedMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T, I, K, O> hoodieTable,
+       Iterator<HoodieRecord<T>> recordItr, String partitionPath, String fileId, TaskContextSupplier taskContextSupplier) {
+    super(config, instantTime, hoodieTable, recordItr, partitionPath, fileId, taskContextSupplier);
     newRecordKeysSorted.addAll(keyToNewRecords.keySet());
   }
 
   /**
    * Called by compactor code path.
    */
-  public HoodieSortedMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
+  public HoodieSortedMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T, I, K, O> hoodieTable,
       Map<String, HoodieRecord<T>> keyToNewRecordsOrig, String partitionPath, String fileId,
-      HoodieBaseFile dataFileToBeMerged, SparkTaskContextSupplier sparkTaskContextSupplier) {
+      HoodieBaseFile dataFileToBeMerged, TaskContextSupplier taskContextSupplier) {
     super(config, instantTime, hoodieTable, keyToNewRecordsOrig, partitionPath, fileId, dataFileToBeMerged,
-        sparkTaskContextSupplier);
+        taskContextSupplier);
 
     newRecordKeysSorted.addAll(keyToNewRecords.keySet());
   }
diff --git a/hudi-client/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java
similarity index 87%
rename from hudi-client/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java
index 5ea8c38..85898bc 100644
--- a/hudi-client/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java
@@ -19,8 +19,8 @@
 package org.apache.hudi.io;
 
 import org.apache.hudi.avro.HoodieAvroUtils;
-import org.apache.hudi.client.SparkTaskContextSupplier;
 import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.TaskContextSupplier;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieRecordPayload;
@@ -49,7 +49,7 @@
 /**
  * Base class for all write operations logically performed at the file group level.
  */
-public abstract class HoodieWriteHandle<T extends HoodieRecordPayload> extends HoodieIOHandle {
+public abstract class HoodieWriteHandle<T extends HoodieRecordPayload, I, K, O> extends HoodieIOHandle<T, I, K, O> {
 
   private static final Logger LOG = LogManager.getLogger(HoodieWriteHandle.class);
 
@@ -60,17 +60,17 @@
   protected final String partitionPath;
   protected final String fileId;
   protected final String writeToken;
-  protected final SparkTaskContextSupplier sparkTaskContextSupplier;
+  protected final TaskContextSupplier taskContextSupplier;
 
   public HoodieWriteHandle(HoodieWriteConfig config, String instantTime, String partitionPath,
-                           String fileId, HoodieTable<T> hoodieTable, SparkTaskContextSupplier sparkTaskContextSupplier) {
+                           String fileId, HoodieTable<T, I, K, O> hoodieTable, TaskContextSupplier taskContextSupplier) {
     this(config, instantTime, partitionPath, fileId, hoodieTable,
-        getWriterSchemaIncludingAndExcludingMetadataPair(config), sparkTaskContextSupplier);
+        getWriterSchemaIncludingAndExcludingMetadataPair(config), taskContextSupplier);
   }
 
   protected HoodieWriteHandle(HoodieWriteConfig config, String instantTime, String partitionPath, String fileId,
-                              HoodieTable<T> hoodieTable, Pair<Schema, Schema> writerSchemaIncludingAndExcludingMetadataPair,
-                              SparkTaskContextSupplier sparkTaskContextSupplier) {
+                              HoodieTable<T, I, K, O> hoodieTable, Pair<Schema, Schema> writerSchemaIncludingAndExcludingMetadataPair,
+                              TaskContextSupplier taskContextSupplier) {
     super(config, instantTime, hoodieTable);
     this.partitionPath = partitionPath;
     this.fileId = fileId;
@@ -79,7 +79,7 @@
     this.timer = new HoodieTimer().startTimer();
     this.writeStatus = (WriteStatus) ReflectionUtils.loadClass(config.getWriteStatusClassName(),
         !hoodieTable.getIndex().isImplicitWithStorage(), config.getWriteStatusFailureFraction());
-    this.sparkTaskContextSupplier = sparkTaskContextSupplier;
+    this.taskContextSupplier = taskContextSupplier;
     this.writeToken = makeWriteToken();
   }
 
@@ -179,19 +179,19 @@
   }
 
   protected int getPartitionId() {
-    return sparkTaskContextSupplier.getPartitionIdSupplier().get();
+    return taskContextSupplier.getPartitionIdSupplier().get();
   }
 
   protected int getStageId() {
-    return sparkTaskContextSupplier.getStageIdSupplier().get();
+    return taskContextSupplier.getStageIdSupplier().get();
   }
 
   protected long getAttemptId() {
-    return sparkTaskContextSupplier.getAttemptIdSupplier().get();
+    return taskContextSupplier.getAttemptIdSupplier().get();
   }
 
-  protected HoodieFileWriter createNewFileWriter(String instantTime, Path path, HoodieTable<T> hoodieTable,
-      HoodieWriteConfig config, Schema schema, SparkTaskContextSupplier sparkTaskContextSupplier) throws IOException {
-    return HoodieFileWriterFactory.getFileWriter(instantTime, path, hoodieTable, config, schema, sparkTaskContextSupplier);
+  protected HoodieFileWriter createNewFileWriter(String instantTime, Path path, HoodieTable<T, I, K, O> hoodieTable,
+      HoodieWriteConfig config, Schema schema, TaskContextSupplier taskContextSupplier) throws IOException {
+    return HoodieFileWriterFactory.getFileWriter(instantTime, path, hoodieTable, config, schema, taskContextSupplier);
   }
 }
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/IOUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/IOUtils.java
new file mode 100644
index 0000000..0371915
--- /dev/null
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/IOUtils.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.hudi.io;
+
+import org.apache.hudi.client.common.EngineProperty;
+import org.apache.hudi.client.common.TaskContextSupplier;
+import org.apache.hudi.common.util.Option;
+
+import java.util.Properties;
+
+import static org.apache.hudi.config.HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES;
+import static org.apache.hudi.config.HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FRACTION_FOR_COMPACTION;
+import static org.apache.hudi.config.HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FRACTION_FOR_MERGE;
+import static org.apache.hudi.config.HoodieMemoryConfig.DEFAULT_MIN_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES;
+import static org.apache.hudi.config.HoodieMemoryConfig.MAX_MEMORY_FOR_COMPACTION_PROP;
+import static org.apache.hudi.config.HoodieMemoryConfig.MAX_MEMORY_FOR_MERGE_PROP;
+import static org.apache.hudi.config.HoodieMemoryConfig.MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP;
+import static org.apache.hudi.config.HoodieMemoryConfig.MAX_MEMORY_FRACTION_FOR_MERGE_PROP;
+
+public class IOUtils {
+  /**
+   * Dynamic calculation of max memory to use for for spillable map. user.available.memory = executor.memory *
+   * (1 - memory.fraction) spillable.available.memory = user.available.memory * hoodie.memory.fraction. Anytime
+   * the engine memory fractions/total memory is changed, the memory used for spillable map changes
+   * accordingly
+   */
+  public static long getMaxMemoryAllowedForMerge(TaskContextSupplier context, String maxMemoryFraction) {
+    Option<String> totalMemoryOpt = context.getProperty(EngineProperty.TOTAL_MEMORY_AVAILABLE);
+    Option<String> memoryFractionOpt = context.getProperty(EngineProperty.MEMORY_FRACTION_IN_USE);
+
+    if (totalMemoryOpt.isPresent() && memoryFractionOpt.isPresent()) {
+      long executorMemoryInBytes = Long.parseLong(totalMemoryOpt.get());
+      double memoryFraction = Double.parseDouble(memoryFractionOpt.get());
+      double maxMemoryFractionForMerge = Double.parseDouble(maxMemoryFraction);
+      double userAvailableMemory = executorMemoryInBytes * (1 - memoryFraction);
+      long maxMemoryForMerge = (long) Math.floor(userAvailableMemory * maxMemoryFractionForMerge);
+      return Math.max(DEFAULT_MIN_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES, maxMemoryForMerge);
+    } else {
+      return DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES;
+    }
+  }
+
+  public static long getMaxMemoryPerPartitionMerge(TaskContextSupplier context, Properties properties) {
+    if (properties.containsKey(MAX_MEMORY_FOR_MERGE_PROP)) {
+      return Long.parseLong(properties.getProperty(MAX_MEMORY_FOR_MERGE_PROP));
+    }
+    String fraction = properties.getProperty(MAX_MEMORY_FRACTION_FOR_MERGE_PROP, DEFAULT_MAX_MEMORY_FRACTION_FOR_MERGE);
+    return getMaxMemoryAllowedForMerge(context, fraction);
+  }
+
+  public static long getMaxMemoryPerCompaction(TaskContextSupplier context, Properties properties) {
+    if (properties.containsKey(MAX_MEMORY_FOR_COMPACTION_PROP)) {
+      return Long.parseLong(properties.getProperty(MAX_MEMORY_FOR_COMPACTION_PROP));
+    }
+    String fraction = properties.getProperty(MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP, DEFAULT_MAX_MEMORY_FRACTION_FOR_COMPACTION);
+    return getMaxMemoryAllowedForMerge(context, fraction);
+  }
+}
diff --git a/hudi-client/src/main/java/org/apache/hudi/io/WriteHandleFactory.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/WriteHandleFactory.java
similarity index 79%
rename from hudi-client/src/main/java/org/apache/hudi/io/WriteHandleFactory.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/WriteHandleFactory.java
index 7039b71..c66442a 100644
--- a/hudi-client/src/main/java/org/apache/hudi/io/WriteHandleFactory.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/WriteHandleFactory.java
@@ -18,16 +18,16 @@
 
 package org.apache.hudi.io;
 
-import org.apache.hudi.client.SparkTaskContextSupplier;
+import org.apache.hudi.client.common.TaskContextSupplier;
 import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.table.HoodieTable;
 
-public abstract class WriteHandleFactory<T extends HoodieRecordPayload> {
+public abstract class WriteHandleFactory<T extends HoodieRecordPayload, I, K, O> {
   private int numFilesWritten = 0;
 
-  public abstract HoodieWriteHandle<T> create(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable,
-      String partitionPath, String fileIdPrefix, SparkTaskContextSupplier sparkTaskContextSupplier);
+  public abstract HoodieWriteHandle<T, I, K, O> create(HoodieWriteConfig config, String commitTime, HoodieTable<T, I, K, O> hoodieTable,
+      String partitionPath, String fileIdPrefix, TaskContextSupplier taskContextSupplier);
 
   protected String getNextFileId(String idPfx) {
     return String.format("%s-%d", idPfx, numFilesWritten++);
diff --git a/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetConfig.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetConfig.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetConfig.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieBaseParquetConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieBaseParquetConfig.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieBaseParquetConfig.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieBaseParquetConfig.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieFileWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriter.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieFileWriter.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriter.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java
similarity index 85%
rename from hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java
index 1d4a9a2..5f4eec0 100644
--- a/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java
@@ -19,7 +19,7 @@
 package org.apache.hudi.io.storage;
 
 import org.apache.hudi.avro.HoodieAvroWriteSupport;
-import org.apache.hudi.client.SparkTaskContextSupplier;
+import org.apache.hudi.client.common.TaskContextSupplier;
 import org.apache.hudi.common.bloom.BloomFilter;
 import org.apache.hudi.common.bloom.BloomFilterFactory;
 import org.apache.hudi.common.fs.FSUtils;
@@ -39,22 +39,22 @@
 
 public class HoodieFileWriterFactory {
 
-  public static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieFileWriter<R> getFileWriter(
-      String instantTime, Path path, HoodieTable<T> hoodieTable, HoodieWriteConfig config, Schema schema,
-      SparkTaskContextSupplier sparkTaskContextSupplier) throws IOException {
+  public static <T extends HoodieRecordPayload, R extends IndexedRecord, I, K, O> HoodieFileWriter<R> getFileWriter(
+      String instantTime, Path path, HoodieTable<T, I, K, O> hoodieTable, HoodieWriteConfig config, Schema schema,
+      TaskContextSupplier taskContextSupplier) throws IOException {
     final String extension = FSUtils.getFileExtension(path.getName());
     if (PARQUET.getFileExtension().equals(extension)) {
-      return newParquetFileWriter(instantTime, path, config, schema, hoodieTable, sparkTaskContextSupplier);
+      return newParquetFileWriter(instantTime, path, config, schema, hoodieTable, taskContextSupplier);
     }
     if (HFILE.getFileExtension().equals(extension)) {
-      return newHFileFileWriter(instantTime, path, config, schema, hoodieTable, sparkTaskContextSupplier);
+      return newHFileFileWriter(instantTime, path, config, schema, hoodieTable, taskContextSupplier);
     }
     throw new UnsupportedOperationException(extension + " format not supported yet.");
   }
 
   private static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieFileWriter<R> newParquetFileWriter(
       String instantTime, Path path, HoodieWriteConfig config, Schema schema, HoodieTable hoodieTable,
-      SparkTaskContextSupplier sparkTaskContextSupplier) throws IOException {
+      TaskContextSupplier taskContextSupplier) throws IOException {
     BloomFilter filter = createBloomFilter(config);
     HoodieAvroWriteSupport writeSupport =
         new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, filter);
@@ -63,18 +63,18 @@
         config.getParquetBlockSize(), config.getParquetPageSize(), config.getParquetMaxFileSize(),
         hoodieTable.getHadoopConf(), config.getParquetCompressionRatio());
 
-    return new HoodieParquetWriter<>(instantTime, path, parquetConfig, schema, sparkTaskContextSupplier);
+    return new HoodieParquetWriter<>(instantTime, path, parquetConfig, schema, taskContextSupplier);
   }
 
   private static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieFileWriter<R> newHFileFileWriter(
       String instantTime, Path path, HoodieWriteConfig config, Schema schema, HoodieTable hoodieTable,
-      SparkTaskContextSupplier sparkTaskContextSupplier) throws IOException {
+      TaskContextSupplier taskContextSupplier) throws IOException {
 
     BloomFilter filter = createBloomFilter(config);
     HoodieHFileConfig hfileConfig = new HoodieHFileConfig(hoodieTable.getHadoopConf(),
         config.getHFileCompressionAlgorithm(), config.getHFileBlockSize(), config.getHFileMaxFileSize(), filter);
 
-    return new HoodieHFileWriter<>(instantTime, path, hfileConfig, schema, sparkTaskContextSupplier);
+    return new HoodieHFileWriter<>(instantTime, path, hfileConfig, schema, taskContextSupplier);
   }
 
   private static BloomFilter createBloomFilter(HoodieWriteConfig config) {
diff --git a/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieHFileConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileConfig.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieHFileConfig.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileConfig.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieHFileWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileWriter.java
similarity index 93%
rename from hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieHFileWriter.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileWriter.java
index c7f549a..3684f9d 100644
--- a/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieHFileWriter.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileWriter.java
@@ -19,7 +19,7 @@
 package org.apache.hudi.io.storage;
 
 import org.apache.hudi.avro.HoodieAvroUtils;
-import org.apache.hudi.client.SparkTaskContextSupplier;
+import org.apache.hudi.client.common.TaskContextSupplier;
 import org.apache.hudi.common.bloom.BloomFilter;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.fs.HoodieWrapperFileSystem;
@@ -60,7 +60,7 @@
   private final HoodieWrapperFileSystem fs;
   private final long maxFileSize;
   private final String instantTime;
-  private final SparkTaskContextSupplier sparkTaskContextSupplier;
+  private final TaskContextSupplier taskContextSupplier;
   private HFile.Writer writer;
   private String minRecordKey;
   private String maxRecordKey;
@@ -69,7 +69,7 @@
   private static String DROP_BEHIND_CACHE_COMPACTION_KEY = "hbase.hfile.drop.behind.compaction";
 
   public HoodieHFileWriter(String instantTime, Path file, HoodieHFileConfig hfileConfig, Schema schema,
-      SparkTaskContextSupplier sparkTaskContextSupplier) throws IOException {
+                           TaskContextSupplier taskContextSupplier) throws IOException {
 
     Configuration conf = FSUtils.registerFileSystem(file, hfileConfig.getHadoopConf());
     this.file = HoodieWrapperFileSystem.convertToHoodiePath(file, conf);
@@ -82,7 +82,7 @@
     //    + Math.round(hfileConfig.getMaxFileSize() * hfileConfig.getCompressionRatio());
     this.maxFileSize = hfileConfig.getMaxFileSize();
     this.instantTime = instantTime;
-    this.sparkTaskContextSupplier = sparkTaskContextSupplier;
+    this.taskContextSupplier = taskContextSupplier;
 
     HFileContext context = new HFileContextBuilder().withBlockSize(hfileConfig.getBlockSize())
           .withCompression(hfileConfig.getCompressionAlgorithm())
@@ -100,7 +100,7 @@
   @Override
   public void writeAvroWithMetadata(R avroRecord, HoodieRecord record) throws IOException {
     String seqId =
-        HoodieRecord.generateSequenceId(instantTime, sparkTaskContextSupplier.getPartitionIdSupplier().get(), recordIndex.getAndIncrement());
+        HoodieRecord.generateSequenceId(instantTime, taskContextSupplier.getPartitionIdSupplier().get(), recordIndex.getAndIncrement());
     HoodieAvroUtils.addHoodieKeyToRecord((GenericRecord) avroRecord, record.getRecordKey(), record.getPartitionPath(),
         file.getName());
     HoodieAvroUtils.addCommitMetadataToRecord((GenericRecord) avroRecord, instantTime, seqId);
diff --git a/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieParquetWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieParquetWriter.java
similarity index 86%
rename from hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieParquetWriter.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieParquetWriter.java
index 5c8f99f..166e2bc 100644
--- a/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieParquetWriter.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieParquetWriter.java
@@ -20,7 +20,7 @@
 
 import org.apache.hudi.avro.HoodieAvroUtils;
 import org.apache.hudi.avro.HoodieAvroWriteSupport;
-import org.apache.hudi.client.SparkTaskContextSupplier;
+import org.apache.hudi.client.common.TaskContextSupplier;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.fs.HoodieWrapperFileSystem;
 import org.apache.hudi.common.model.HoodieRecord;
@@ -50,15 +50,15 @@
   private final long maxFileSize;
   private final HoodieAvroWriteSupport writeSupport;
   private final String instantTime;
-  private final SparkTaskContextSupplier sparkTaskContextSupplier;
+  private final TaskContextSupplier taskContextSupplier;
 
   public HoodieParquetWriter(String instantTime, Path file, HoodieAvroParquetConfig parquetConfig,
-      Schema schema, SparkTaskContextSupplier sparkTaskContextSupplier) throws IOException {
+      Schema schema, TaskContextSupplier taskContextSupplier) throws IOException {
     super(HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf()),
         ParquetFileWriter.Mode.CREATE, parquetConfig.getWriteSupport(), parquetConfig.getCompressionCodecName(),
         parquetConfig.getBlockSize(), parquetConfig.getPageSize(), parquetConfig.getPageSize(),
-        ParquetWriter.DEFAULT_IS_DICTIONARY_ENABLED, ParquetWriter.DEFAULT_IS_VALIDATING_ENABLED,
-        ParquetWriter.DEFAULT_WRITER_VERSION, FSUtils.registerFileSystem(file, parquetConfig.getHadoopConf()));
+        DEFAULT_IS_DICTIONARY_ENABLED, DEFAULT_IS_VALIDATING_ENABLED,
+        DEFAULT_WRITER_VERSION, FSUtils.registerFileSystem(file, parquetConfig.getHadoopConf()));
     this.file = HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf());
     this.fs =
         (HoodieWrapperFileSystem) this.file.getFileSystem(FSUtils.registerFileSystem(file, parquetConfig.getHadoopConf()));
@@ -70,13 +70,13 @@
         + Math.round(parquetConfig.getMaxFileSize() * parquetConfig.getCompressionRatio());
     this.writeSupport = parquetConfig.getWriteSupport();
     this.instantTime = instantTime;
-    this.sparkTaskContextSupplier = sparkTaskContextSupplier;
+    this.taskContextSupplier = taskContextSupplier;
   }
 
   @Override
   public void writeAvroWithMetadata(R avroRecord, HoodieRecord record) throws IOException {
     String seqId =
-        HoodieRecord.generateSequenceId(instantTime, sparkTaskContextSupplier.getPartitionIdSupplier().get(), recordIndex.getAndIncrement());
+        HoodieRecord.generateSequenceId(instantTime, taskContextSupplier.getPartitionIdSupplier().get(), recordIndex.getAndIncrement());
     HoodieAvroUtils.addHoodieKeyToRecord((GenericRecord) avroRecord, record.getRecordKey(), record.getPartitionPath(),
         file.getName());
     HoodieAvroUtils.addCommitMetadataToRecord((GenericRecord) avroRecord, instantTime, seqId);
diff --git a/hudi-client/src/main/java/org/apache/hudi/keygen/KeyGeneratorInterface.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/KeyGeneratorInterface.java
similarity index 91%
rename from hudi-client/src/main/java/org/apache/hudi/keygen/KeyGeneratorInterface.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/KeyGeneratorInterface.java
index 6412a2f..fc8fee2 100644
--- a/hudi-client/src/main/java/org/apache/hudi/keygen/KeyGeneratorInterface.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/KeyGeneratorInterface.java
@@ -20,7 +20,6 @@
 
 import org.apache.avro.generic.GenericRecord;
 import org.apache.hudi.common.model.HoodieKey;
-import org.apache.spark.sql.Row;
 
 import java.io.Serializable;
 import java.util.List;
@@ -34,8 +33,4 @@
 
   List<String> getRecordKeyFieldNames();
 
-  String getRecordKey(Row row);
-
-  String getPartitionPath(Row row);
-
 }
diff --git a/hudi-client/src/main/java/org/apache/hudi/metrics/ConsoleMetricsReporter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/ConsoleMetricsReporter.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/metrics/ConsoleMetricsReporter.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/ConsoleMetricsReporter.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/metrics/HoodieGauge.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/HoodieGauge.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/metrics/HoodieGauge.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/HoodieGauge.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/metrics/HoodieMetrics.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/HoodieMetrics.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/metrics/HoodieMetrics.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/HoodieMetrics.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/metrics/InMemoryMetricsReporter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/InMemoryMetricsReporter.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/metrics/InMemoryMetricsReporter.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/InMemoryMetricsReporter.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/metrics/JmxMetricsReporter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/JmxMetricsReporter.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/metrics/JmxMetricsReporter.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/JmxMetricsReporter.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/metrics/JmxReporterServer.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/JmxReporterServer.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/metrics/JmxReporterServer.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/JmxReporterServer.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/metrics/Metrics.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/Metrics.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/metrics/Metrics.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/Metrics.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/metrics/MetricsGraphiteReporter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/MetricsGraphiteReporter.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/metrics/MetricsGraphiteReporter.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/MetricsGraphiteReporter.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/metrics/MetricsReporter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/MetricsReporter.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/metrics/MetricsReporter.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/MetricsReporter.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/metrics/MetricsReporterFactory.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/MetricsReporterFactory.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/metrics/MetricsReporterFactory.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/MetricsReporterFactory.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/metrics/MetricsReporterType.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/MetricsReporterType.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/metrics/MetricsReporterType.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/MetricsReporterType.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/metrics/datadog/DatadogHttpClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/datadog/DatadogHttpClient.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/metrics/datadog/DatadogHttpClient.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/datadog/DatadogHttpClient.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/metrics/datadog/DatadogMetricsReporter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/datadog/DatadogMetricsReporter.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/metrics/datadog/DatadogMetricsReporter.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/datadog/DatadogMetricsReporter.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/metrics/datadog/DatadogReporter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/datadog/DatadogReporter.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/metrics/datadog/DatadogReporter.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/datadog/DatadogReporter.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/metrics/prometheus/PrometheusReporter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/prometheus/PrometheusReporter.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/metrics/prometheus/PrometheusReporter.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/prometheus/PrometheusReporter.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/metrics/prometheus/PushGatewayMetricsReporter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/prometheus/PushGatewayMetricsReporter.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/metrics/prometheus/PushGatewayMetricsReporter.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/prometheus/PushGatewayMetricsReporter.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/metrics/prometheus/PushGatewayReporter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/prometheus/PushGatewayReporter.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/metrics/prometheus/PushGatewayReporter.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/prometheus/PushGatewayReporter.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/metrics/userdefined/AbstractUserDefinedMetricsReporter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/userdefined/AbstractUserDefinedMetricsReporter.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/metrics/userdefined/AbstractUserDefinedMetricsReporter.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metrics/userdefined/AbstractUserDefinedMetricsReporter.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/BulkInsertPartitioner.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/BulkInsertPartitioner.java
similarity index 75%
rename from hudi-client/src/main/java/org/apache/hudi/table/BulkInsertPartitioner.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/BulkInsertPartitioner.java
index 53aee2f..b571fd9 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/BulkInsertPartitioner.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/BulkInsertPartitioner.java
@@ -18,26 +18,21 @@
 
 package org.apache.hudi.table;
 
-import org.apache.hudi.common.model.HoodieRecord;
-import org.apache.hudi.common.model.HoodieRecordPayload;
-
-import org.apache.spark.api.java.JavaRDD;
-
 /**
  * Repartition input records into at least expected number of output spark partitions. It should give below guarantees -
  * Output spark partition will have records from only one hoodie partition. - Average records per output spark
  * partitions should be almost equal to (#inputRecords / #outputSparkPartitions) to avoid possible skews.
  */
-public interface BulkInsertPartitioner<T extends HoodieRecordPayload> {
+public interface BulkInsertPartitioner<I> {
 
   /**
    * Repartitions the input records into at least expected number of output spark partitions.
    *
-   * @param records               Input Hoodie records in RDD
-   * @param outputSparkPartitions Expected number of output RDD partitions
+   * @param records               Input Hoodie records
+   * @param outputSparkPartitions Expected number of output partitions
    * @return
    */
-  JavaRDD<HoodieRecord<T>> repartitionRecords(JavaRDD<HoodieRecord<T>> records, int outputSparkPartitions);
+  I repartitionRecords(I records, int outputSparkPartitions);
 
   /**
    * @return {@code true} if the records within a RDD partition are sorted; {@code false} otherwise.
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/HoodieTable.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java
similarity index 72%
rename from hudi-client/src/main/java/org/apache/hudi/table/HoodieTable.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java
index 5c824a6..afd1a36 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/HoodieTable.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java
@@ -28,7 +28,8 @@
 import org.apache.hudi.avro.model.HoodieRestoreMetadata;
 import org.apache.hudi.avro.model.HoodieRollbackMetadata;
 import org.apache.hudi.avro.model.HoodieSavepointMetadata;
-import org.apache.hudi.client.SparkTaskContextSupplier;
+import org.apache.hudi.client.common.TaskContextSupplier;
+import org.apache.hudi.client.common.HoodieEngineContext;
 import org.apache.hudi.common.config.SerializableConfiguration;
 import org.apache.hudi.common.fs.ConsistencyGuard;
 import org.apache.hudi.common.fs.ConsistencyGuard.FileVisibility;
@@ -37,7 +38,6 @@
 import org.apache.hudi.common.fs.OptimisticConsistencyGuard;
 import org.apache.hudi.common.model.HoodieFileFormat;
 import org.apache.hudi.common.model.HoodieKey;
-import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.model.HoodieWriteStat;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
@@ -46,7 +46,6 @@
 import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
 import org.apache.hudi.common.table.timeline.HoodieInstant;
 import org.apache.hudi.common.table.timeline.HoodieTimeline;
-import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
 import org.apache.hudi.common.table.view.FileSystemViewManager;
 import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
 import org.apache.hudi.common.table.view.SyncableFileSystemView;
@@ -65,8 +64,6 @@
 import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
-import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
 
 import java.io.IOException;
 import java.io.Serializable;
@@ -80,29 +77,37 @@
 
 /**
  * Abstract implementation of a HoodieTable.
+ *
+ * @param <T> Sub type of HoodieRecordPayload
+ * @param <I> Type of inputs
+ * @param <K> Type of keys
+ * @param <O> Type of outputs
  */
-public abstract class HoodieTable<T extends HoodieRecordPayload> implements Serializable {
+public abstract class HoodieTable<T extends HoodieRecordPayload, I, K, O> implements Serializable {
 
   private static final Logger LOG = LogManager.getLogger(HoodieTable.class);
 
   protected final HoodieWriteConfig config;
   protected final HoodieTableMetaClient metaClient;
-  protected final HoodieIndex<T> index;
+  protected final HoodieIndex<T, I, K, O> index;
 
   private SerializableConfiguration hadoopConfiguration;
   private transient FileSystemViewManager viewManager;
 
-  protected final SparkTaskContextSupplier sparkTaskContextSupplier = new SparkTaskContextSupplier();
+  protected final TaskContextSupplier taskContextSupplier;
 
-  protected HoodieTable(HoodieWriteConfig config, Configuration hadoopConf, HoodieTableMetaClient metaClient) {
+  protected HoodieTable(HoodieWriteConfig config, HoodieEngineContext context, HoodieTableMetaClient metaClient) {
     this.config = config;
-    this.hadoopConfiguration = new SerializableConfiguration(hadoopConf);
-    this.viewManager = FileSystemViewManager.createViewManager(new SerializableConfiguration(hadoopConf),
+    this.hadoopConfiguration = context.getHadoopConf();
+    this.viewManager = FileSystemViewManager.createViewManager(hadoopConfiguration,
         config.getViewStorageConfig());
     this.metaClient = metaClient;
-    this.index = HoodieIndex.createIndex(config);
+    this.index = getIndex(config);
+    this.taskContextSupplier = context.getTaskContextSupplier();
   }
 
+  protected abstract HoodieIndex<T, I, K, O> getIndex(HoodieWriteConfig config);
+
   private synchronized FileSystemViewManager getViewManager() {
     if (null == viewManager) {
       viewManager = FileSystemViewManager.createViewManager(hadoopConfiguration, config.getViewStorageConfig());
@@ -110,114 +115,95 @@
     return viewManager;
   }
 
-  public static <T extends HoodieRecordPayload> HoodieTable<T> create(HoodieWriteConfig config, Configuration hadoopConf) {
-    HoodieTableMetaClient metaClient = new HoodieTableMetaClient(
-        hadoopConf,
-        config.getBasePath(),
-        true,
-        config.getConsistencyGuardConfig(),
-        Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion()))
-    );
-    return HoodieTable.create(metaClient, config, hadoopConf);
-  }
-
-  public static <T extends HoodieRecordPayload> HoodieTable<T> create(HoodieTableMetaClient metaClient,
-                                                                      HoodieWriteConfig config,
-                                                                      Configuration hadoopConf) {
-    switch (metaClient.getTableType()) {
-      case COPY_ON_WRITE:
-        return new HoodieCopyOnWriteTable<>(config, hadoopConf, metaClient);
-      case MERGE_ON_READ:
-        return new HoodieMergeOnReadTable<>(config, hadoopConf, metaClient);
-      default:
-        throw new HoodieException("Unsupported table type :" + metaClient.getTableType());
-    }
-  }
-
   /**
    * Upsert a batch of new records into Hoodie table at the supplied instantTime.
-   * @param jsc    Java Spark Context jsc
+   * @param context    HoodieEngineContext
    * @param instantTime Instant Time for the action
-   * @param records  JavaRDD of hoodieRecords to upsert
+   * @param records  hoodieRecords to upsert
    * @return HoodieWriteMetadata
    */
-  public abstract HoodieWriteMetadata upsert(JavaSparkContext jsc, String instantTime,
-      JavaRDD<HoodieRecord<T>> records);
+  public abstract HoodieWriteMetadata<O> upsert(HoodieEngineContext context, String instantTime,
+      I records);
 
   /**
    * Insert a batch of new records into Hoodie table at the supplied instantTime.
-   * @param jsc    Java Spark Context jsc
+   * @param context    HoodieEngineContext
    * @param instantTime Instant Time for the action
-   * @param records  JavaRDD of hoodieRecords to upsert
+   * @param records  hoodieRecords to upsert
    * @return HoodieWriteMetadata
    */
-  public abstract HoodieWriteMetadata insert(JavaSparkContext jsc, String instantTime,
-      JavaRDD<HoodieRecord<T>> records);
+  public abstract HoodieWriteMetadata<O> insert(HoodieEngineContext context, String instantTime,
+      I records);
 
   /**
    * Bulk Insert a batch of new records into Hoodie table at the supplied instantTime.
-   * @param jsc    Java Spark Context jsc
+   * @param context    HoodieEngineContext
    * @param instantTime Instant Time for the action
-   * @param records  JavaRDD of hoodieRecords to upsert
+   * @param records  hoodieRecords to upsert
    * @param bulkInsertPartitioner User Defined Partitioner
    * @return HoodieWriteMetadata
    */
-  public abstract HoodieWriteMetadata bulkInsert(JavaSparkContext jsc, String instantTime,
-      JavaRDD<HoodieRecord<T>> records, Option<BulkInsertPartitioner> bulkInsertPartitioner);
+  public abstract HoodieWriteMetadata<O> bulkInsert(HoodieEngineContext context, String instantTime,
+      I records, Option<BulkInsertPartitioner<I>> bulkInsertPartitioner);
 
   /**
    * Deletes a list of {@link HoodieKey}s from the Hoodie table, at the supplied instantTime {@link HoodieKey}s will be
    * de-duped and non existent keys will be removed before deleting.
    *
-   * @param jsc    Java Spark Context jsc
+   * @param context    HoodieEngineContext
    * @param instantTime Instant Time for the action
    * @param keys   {@link List} of {@link HoodieKey}s to be deleted
    * @return HoodieWriteMetadata
    */
-  public abstract HoodieWriteMetadata delete(JavaSparkContext jsc, String instantTime, JavaRDD<HoodieKey> keys);
+  public abstract HoodieWriteMetadata<O> delete(HoodieEngineContext context, String instantTime, K keys);
 
   /**
    * Upserts the given prepared records into the Hoodie table, at the supplied instantTime.
    * <p>
    * This implementation requires that the input records are already tagged, and de-duped if needed.
-   * @param jsc    Java Spark Context jsc
+   * @param context    HoodieEngineContext
    * @param instantTime Instant Time for the action
-   * @param preppedRecords  JavaRDD of hoodieRecords to upsert
+   * @param preppedRecords  hoodieRecords to upsert
    * @return HoodieWriteMetadata
    */
-  public abstract HoodieWriteMetadata upsertPrepped(JavaSparkContext jsc, String instantTime,
-      JavaRDD<HoodieRecord<T>> preppedRecords);
+  public abstract HoodieWriteMetadata<O> upsertPrepped(HoodieEngineContext context, String instantTime,
+      I preppedRecords);
 
   /**
    * Inserts the given prepared records into the Hoodie table, at the supplied instantTime.
    * <p>
    * This implementation requires that the input records are already tagged, and de-duped if needed.
-   * @param jsc    Java Spark Context jsc
+   * @param context    HoodieEngineContext
    * @param instantTime Instant Time for the action
-   * @param preppedRecords  JavaRDD of hoodieRecords to upsert
+   * @param preppedRecords  hoodieRecords to upsert
    * @return HoodieWriteMetadata
    */
-  public abstract HoodieWriteMetadata insertPrepped(JavaSparkContext jsc, String instantTime,
-      JavaRDD<HoodieRecord<T>> preppedRecords);
+  public abstract HoodieWriteMetadata<O> insertPrepped(HoodieEngineContext context, String instantTime,
+      I preppedRecords);
 
   /**
    * Bulk Insert the given prepared records into the Hoodie table, at the supplied instantTime.
    * <p>
    * This implementation requires that the input records are already tagged, and de-duped if needed.
-   * @param jsc    Java Spark Context jsc
+   * @param context    HoodieEngineContext
    * @param instantTime Instant Time for the action
-   * @param preppedRecords  JavaRDD of hoodieRecords to upsert
+   * @param preppedRecords  hoodieRecords to upsert
    * @param bulkInsertPartitioner User Defined Partitioner
    * @return HoodieWriteMetadata
    */
-  public abstract HoodieWriteMetadata bulkInsertPrepped(JavaSparkContext jsc, String instantTime,
-      JavaRDD<HoodieRecord<T>> preppedRecords,  Option<BulkInsertPartitioner> bulkInsertPartitioner);
+  public abstract HoodieWriteMetadata<O> bulkInsertPrepped(HoodieEngineContext context, String instantTime,
+      I preppedRecords,  Option<BulkInsertPartitioner<I>> bulkInsertPartitioner);
 
   /**
-   * Logically delete all existing records and Insert a batch of new records into Hoodie table at the supplied instantTime.
+   * Replaces all the existing records and inserts the specified new records into Hoodie table at the supplied instantTime,
+   * for the partition paths contained in input records.
+   *
+   * @param context HoodieEngineContext
+   * @param instantTime Instant time for the replace action
+   * @param records input records
+   * @return HoodieWriteMetadata
    */
-  public abstract HoodieWriteMetadata insertOverwrite(JavaSparkContext jsc, String instantTime,
-                                                      JavaRDD<HoodieRecord<T>> records);
+  public abstract HoodieWriteMetadata<O> insertOverwrite(HoodieEngineContext context, String instantTime, I records);
 
   public HoodieWriteConfig getConfig() {
     return config;
@@ -315,51 +301,51 @@
   /**
    * Return the index.
    */
-  public HoodieIndex<T> getIndex() {
+  public HoodieIndex<T, I, K, O> getIndex() {
     return index;
   }
 
   /**
    * Schedule compaction for the instant time.
    *
-   * @param jsc Spark Context
+   * @param context HoodieEngineContext
    * @param instantTime Instant Time for scheduling compaction
    * @param extraMetadata additional metadata to write into plan
    * @return
    */
-  public abstract Option<HoodieCompactionPlan> scheduleCompaction(JavaSparkContext jsc,
+  public abstract Option<HoodieCompactionPlan> scheduleCompaction(HoodieEngineContext context,
                                                                   String instantTime,
                                                                   Option<Map<String, String>> extraMetadata);
 
   /**
    * Run Compaction on the table. Compaction arranges the data so that it is optimized for data access.
    *
-   * @param jsc Spark Context
+   * @param context HoodieEngineContext
    * @param compactionInstantTime Instant Time
    */
-  public abstract HoodieWriteMetadata compact(JavaSparkContext jsc,
+  public abstract HoodieWriteMetadata<O> compact(HoodieEngineContext context,
                                               String compactionInstantTime);
 
   /**
    * Perform metadata/full bootstrap of a Hudi table.
-   * @param jsc JavaSparkContext
+   * @param context HoodieEngineContext
    * @param extraMetadata Additional Metadata for storing in commit file.
    * @return HoodieBootstrapWriteMetadata
    */
-  public abstract HoodieBootstrapWriteMetadata bootstrap(JavaSparkContext jsc, Option<Map<String, String>> extraMetadata);
+  public abstract HoodieBootstrapWriteMetadata<O> bootstrap(HoodieEngineContext context, Option<Map<String, String>> extraMetadata);
 
   /**
    * Perform rollback of bootstrap of a Hudi table.
-   * @param jsc JavaSparkContext
+   * @param context HoodieEngineContext
    */
-  public abstract void rollbackBootstrap(JavaSparkContext jsc, String instantTime);
+  public abstract void rollbackBootstrap(HoodieEngineContext context, String instantTime);
 
   /**
    * Executes a new clean action.
    *
    * @return information on cleaned file slices
    */
-  public abstract HoodieCleanMetadata clean(JavaSparkContext jsc, String cleanInstantTime);
+  public abstract HoodieCleanMetadata clean(HoodieEngineContext context, String cleanInstantTime);
 
   /**
    * Rollback the (inflight/committed) record changes with the given commit time.
@@ -371,7 +357,7 @@
    *   (4) Finally delete .commit or .inflight file, if deleteInstants = true
    * </pre>
    */
-  public abstract HoodieRollbackMetadata rollback(JavaSparkContext jsc,
+  public abstract HoodieRollbackMetadata rollback(HoodieEngineContext context,
                                                   String rollbackInstantTime,
                                                   HoodieInstant commitInstant,
                                                   boolean deleteInstants);
@@ -380,7 +366,7 @@
    * Create a savepoint at the specified instant, so that the table can be restored
    * to this point-in-timeline later if needed.
    */
-  public abstract HoodieSavepointMetadata savepoint(JavaSparkContext jsc,
+  public abstract HoodieSavepointMetadata savepoint(HoodieEngineContext context,
                                                     String instantToSavepoint,
                                                     String user,
                                                     String comment);
@@ -389,54 +375,53 @@
    * Restore the table to the given instant. Note that this is a admin table recovery operation
    * that would cause any running queries that are accessing file slices written after the instant to fail.
    */
-  public abstract HoodieRestoreMetadata restore(JavaSparkContext jsc,
+  public abstract HoodieRestoreMetadata restore(HoodieEngineContext context,
                                                 String restoreInstantTime,
                                                 String instantToRestore);
 
   /**
    * Finalize the written data onto storage. Perform any final cleanups.
    *
-   * @param jsc Spark Context
+   * @param context HoodieEngineContext
    * @param stats List of HoodieWriteStats
    * @throws HoodieIOException if some paths can't be finalized on storage
    */
-  public void finalizeWrite(JavaSparkContext jsc, String instantTs, List<HoodieWriteStat> stats) throws HoodieIOException {
-    reconcileAgainstMarkers(jsc, instantTs, stats, config.getConsistencyGuardConfig().isConsistencyCheckEnabled());
+  public void finalizeWrite(HoodieEngineContext context, String instantTs, List<HoodieWriteStat> stats) throws HoodieIOException {
+    reconcileAgainstMarkers(context, instantTs, stats, config.getConsistencyGuardConfig().isConsistencyCheckEnabled());
   }
 
-  private void deleteInvalidFilesByPartitions(JavaSparkContext jsc, Map<String, List<Pair<String, String>>> invalidFilesByPartition) {
+  private void deleteInvalidFilesByPartitions(HoodieEngineContext context, Map<String, List<Pair<String, String>>> invalidFilesByPartition) {
     // Now delete partially written files
-    jsc.parallelize(new ArrayList<>(invalidFilesByPartition.values()), config.getFinalizeWriteParallelism())
-        .map(partitionWithFileList -> {
-          final FileSystem fileSystem = metaClient.getFs();
-          LOG.info("Deleting invalid data files=" + partitionWithFileList);
-          if (partitionWithFileList.isEmpty()) {
-            return true;
-          }
-          // Delete
-          partitionWithFileList.stream().map(Pair::getValue).forEach(file -> {
-            try {
-              fileSystem.delete(new Path(file), false);
-            } catch (IOException e) {
-              throw new HoodieIOException(e.getMessage(), e);
-            }
-          });
+    context.map(new ArrayList<>(invalidFilesByPartition.values()), partitionWithFileList -> {
+      final FileSystem fileSystem = metaClient.getFs();
+      LOG.info("Deleting invalid data files=" + partitionWithFileList);
+      if (partitionWithFileList.isEmpty()) {
+        return true;
+      }
+      // Delete
+      partitionWithFileList.stream().map(Pair::getValue).forEach(file -> {
+        try {
+          fileSystem.delete(new Path(file), false);
+        } catch (IOException e) {
+          throw new HoodieIOException(e.getMessage(), e);
+        }
+      });
 
-          return true;
-        }).collect();
+      return true;
+    }, config.getFinalizeWriteParallelism());
   }
 
   /**
    * Reconciles WriteStats and marker files to detect and safely delete duplicate data files created because of Spark
    * retries.
    *
-   * @param jsc Spark Context
+   * @param context HoodieEngineContext
    * @param instantTs Instant Timestamp
    * @param stats Hoodie Write Stat
    * @param consistencyCheckEnabled Consistency Check Enabled
    * @throws HoodieIOException
    */
-  protected void reconcileAgainstMarkers(JavaSparkContext jsc,
+  protected void reconcileAgainstMarkers(HoodieEngineContext context,
                                          String instantTs,
                                          List<HoodieWriteStat> stats,
                                          boolean consistencyCheckEnabled) throws HoodieIOException {
@@ -452,7 +437,7 @@
       }
 
       // we are not including log appends here, since they are already fail-safe.
-      Set<String> invalidDataPaths = markers.createdAndMergedDataPaths(jsc, config.getFinalizeWriteParallelism());
+      Set<String> invalidDataPaths = markers.createdAndMergedDataPaths(context, config.getFinalizeWriteParallelism());
       Set<String> validDataPaths = stats.stream()
           .map(HoodieWriteStat::getPath)
           .filter(p -> p.endsWith(this.getBaseFileExtension()))
@@ -471,17 +456,17 @@
         // Otherwise, we may miss deleting such files. If files are not found even after retries, fail the commit
         if (consistencyCheckEnabled) {
           // This will either ensure all files to be deleted are present.
-          waitForAllFiles(jsc, invalidPathsByPartition, FileVisibility.APPEAR);
+          waitForAllFiles(context, invalidPathsByPartition, FileVisibility.APPEAR);
         }
 
         // Now delete partially written files
-        jsc.setJobGroup(this.getClass().getSimpleName(), "Delete all partially written files");
-        deleteInvalidFilesByPartitions(jsc, invalidPathsByPartition);
+        context.setJobStatus(this.getClass().getSimpleName(), "Delete all partially written files");
+        deleteInvalidFilesByPartitions(context, invalidPathsByPartition);
 
         // Now ensure the deleted files disappear
         if (consistencyCheckEnabled) {
           // This will either ensure all files to be deleted are absent.
-          waitForAllFiles(jsc, invalidPathsByPartition, FileVisibility.DISAPPEAR);
+          waitForAllFiles(context, invalidPathsByPartition, FileVisibility.DISAPPEAR);
         }
       }
     } catch (IOException ioe) {
@@ -492,18 +477,17 @@
   /**
    * Ensures all files passed either appear or disappear.
    *
-   * @param jsc JavaSparkContext
+   * @param context HoodieEngineContext
    * @param groupByPartition Files grouped by partition
    * @param visibility Appear/Disappear
    */
-  private void waitForAllFiles(JavaSparkContext jsc, Map<String, List<Pair<String, String>>> groupByPartition, FileVisibility visibility) {
+  private void waitForAllFiles(HoodieEngineContext context, Map<String, List<Pair<String, String>>> groupByPartition, FileVisibility visibility) {
     // This will either ensure all files to be deleted are present.
-    jsc.setJobGroup(this.getClass().getSimpleName(), "Wait for all files to appear/disappear");
+    context.setJobStatus(this.getClass().getSimpleName(), "Wait for all files to appear/disappear");
     boolean checkPassed =
-        jsc.parallelize(new ArrayList<>(groupByPartition.entrySet()), config.getFinalizeWriteParallelism())
-            .map(partitionWithFileList -> waitForCondition(partitionWithFileList.getKey(),
-                partitionWithFileList.getValue().stream(), visibility))
-            .collect().stream().allMatch(x -> x);
+        context.map(new ArrayList<>(groupByPartition.entrySet()), partitionWithFileList -> waitForCondition(partitionWithFileList.getKey(),
+            partitionWithFileList.getValue().stream(), visibility), config.getFinalizeWriteParallelism())
+            .stream().allMatch(x -> x);
     if (!checkPassed) {
       throw new HoodieIOException("Consistency check failed to ensure all files " + visibility);
     }
@@ -535,8 +519,8 @@
     }
   }
 
-  public SparkTaskContextSupplier getSparkTaskContextSupplier() {
-    return sparkTaskContextSupplier;
+  public TaskContextSupplier getTaskContextSupplier() {
+    return taskContextSupplier;
   }
 
   /**
@@ -544,7 +528,6 @@
    *
    * When inserting/updating data, we read records using the last used schema and convert them to the
    * GenericRecords with writerSchema. Hence, we need to ensure that this conversion can take place without errors.
-   *
    */
   private void validateSchema() throws HoodieUpsertException, HoodieInsertException {
 
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java
similarity index 93%
rename from hudi-client/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java
index 5ce0aed..af2c278 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java
@@ -22,14 +22,16 @@
 import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.avro.Schema;
 import org.apache.avro.generic.IndexedRecord;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hudi.avro.model.HoodieArchivedMetaEntry;
 import org.apache.hudi.avro.model.HoodieCompactionPlan;
 import org.apache.hudi.avro.model.HoodieRollbackMetadata;
 import org.apache.hudi.avro.model.HoodieSavepointMetadata;
+import org.apache.hudi.client.ReplaceArchivalHelper;
+import org.apache.hudi.client.common.HoodieEngineContext;
 import org.apache.hudi.common.model.ActionType;
 import org.apache.hudi.common.model.HoodieArchivedLogFile;
+import org.apache.hudi.common.model.HoodieAvroPayload;
 import org.apache.hudi.common.model.HoodieCommitMetadata;
 import org.apache.hudi.common.model.HoodieReplaceCommitMetadata;
 import org.apache.hudi.common.model.HoodieRollingStatMetadata;
@@ -55,7 +57,6 @@
 import org.apache.hudi.exception.HoodieIOException;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
-import org.apache.spark.api.java.JavaSparkContext;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -75,7 +76,7 @@
 /**
  * Archiver to bound the growth of files under .hoodie meta path.
  */
-public class HoodieTimelineArchiveLog {
+public class HoodieTimelineArchiveLog<T extends HoodieAvroPayload, I, K, O> {
 
   private static final Logger LOG = LogManager.getLogger(HoodieTimelineArchiveLog.class);
 
@@ -84,12 +85,12 @@
   private Writer writer;
   private final int maxInstantsToKeep;
   private final int minInstantsToKeep;
-  private final HoodieTable<?> table;
+  private final HoodieTable<T, I, K, O> table;
   private final HoodieTableMetaClient metaClient;
 
-  public HoodieTimelineArchiveLog(HoodieWriteConfig config, Configuration configuration) {
+  public HoodieTimelineArchiveLog(HoodieWriteConfig config, HoodieTable<T, I, K, O> table) {
     this.config = config;
-    this.table = HoodieTable.create(config, configuration);
+    this.table = table;
     this.metaClient = table.getMetaClient();
     this.archiveFilePath = HoodieArchivedTimeline.getArchiveLogPath(metaClient.getArchivePath());
     this.maxInstantsToKeep = config.getMaxCommitsToKeep();
@@ -123,7 +124,7 @@
   /**
    * Check if commits need to be archived. If yes, archive commits.
    */
-  public boolean archiveIfRequired(JavaSparkContext jsc) throws IOException {
+  public boolean archiveIfRequired(HoodieEngineContext context) throws IOException {
     try {
       List<HoodieInstant> instantsToArchive = getInstantsToArchive().collect(Collectors.toList());
 
@@ -131,7 +132,7 @@
       if (!instantsToArchive.isEmpty()) {
         this.writer = openWriter();
         LOG.info("Archiving instants " + instantsToArchive);
-        archive(jsc, instantsToArchive);
+        archive(context, instantsToArchive);
         LOG.info("Deleting archived instants " + instantsToArchive);
         success = deleteArchivedInstants(instantsToArchive);
       } else {
@@ -269,20 +270,20 @@
     return success;
   }
 
-  public void archive(JavaSparkContext jsc, List<HoodieInstant> instants) throws HoodieCommitException {
+  public void archive(HoodieEngineContext context, List<HoodieInstant> instants) throws HoodieCommitException {
     try {
       HoodieTimeline commitTimeline = metaClient.getActiveTimeline().getAllCommitsTimeline().filterCompletedInstants();
       Schema wrapperSchema = HoodieArchivedMetaEntry.getClassSchema();
       LOG.info("Wrapper schema " + wrapperSchema.toString());
       List<IndexedRecord> records = new ArrayList<>();
       for (HoodieInstant hoodieInstant : instants) {
-        boolean deleteSuccess = deleteReplacedFileGroups(jsc, hoodieInstant);
+        boolean deleteSuccess = deleteReplacedFileGroups(context, hoodieInstant);
         if (!deleteSuccess) {
           // throw error and stop archival if deleting replaced file groups failed.
           throw new HoodieCommitException("Unable to delete file(s) for " + hoodieInstant.getFileName());
         }
         try {
-          deleteAnyLeftOverMarkerFiles(jsc, hoodieInstant);
+          deleteAnyLeftOverMarkerFiles(context, hoodieInstant);
           records.add(convertToAvroRecord(commitTimeline, hoodieInstant));
           if (records.size() >= this.config.getCommitArchivalBatchSize()) {
             writeToFile(wrapperSchema, records);
@@ -300,14 +301,14 @@
     }
   }
 
-  private void deleteAnyLeftOverMarkerFiles(JavaSparkContext jsc, HoodieInstant instant) {
+  private void deleteAnyLeftOverMarkerFiles(HoodieEngineContext context, HoodieInstant instant) {
     MarkerFiles markerFiles = new MarkerFiles(table, instant.getTimestamp());
-    if (markerFiles.deleteMarkerDir(jsc, config.getMarkersDeleteParallelism())) {
+    if (markerFiles.deleteMarkerDir(context, config.getMarkersDeleteParallelism())) {
       LOG.info("Cleaned up left over marker directory for instant :" + instant);
     }
   }
 
-  private boolean deleteReplacedFileGroups(JavaSparkContext jsc, HoodieInstant instant) {
+  private boolean deleteReplacedFileGroups(HoodieEngineContext context, HoodieInstant instant) {
     if (!instant.isCompleted() || !HoodieTimeline.REPLACE_COMMIT_ACTION.equals(instant.getAction())) {
       // only delete files for completed replace instants
       return true;
@@ -315,7 +316,7 @@
 
     TableFileSystemView fileSystemView = this.table.getFileSystemView();
     List<String> replacedPartitions = getReplacedPartitions(instant);
-    return ReplaceArchivalHelper.deleteReplacedFileGroups(jsc, metaClient, fileSystemView, instant, replacedPartitions);
+    return ReplaceArchivalHelper.deleteReplacedFileGroups(context, metaClient, fileSystemView, instant, replacedPartitions);
   }
 
   private List<String> getReplacedPartitions(HoodieInstant instant) {
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/MarkerFiles.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/MarkerFiles.java
similarity index 90%
rename from hudi-client/src/main/java/org/apache/hudi/table/MarkerFiles.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/MarkerFiles.java
index 9577cea..8826204 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/MarkerFiles.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/MarkerFiles.java
@@ -18,6 +18,7 @@
 
 package org.apache.hudi.table;
 
+import org.apache.hudi.client.common.HoodieEngineContext;
 import org.apache.hudi.common.config.SerializableConfiguration;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.IOType;
@@ -33,7 +34,6 @@
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
-import org.apache.spark.api.java.JavaSparkContext;
 
 import java.io.IOException;
 import java.io.Serializable;
@@ -63,16 +63,16 @@
     this.basePath = basePath;
   }
 
-  public MarkerFiles(HoodieTable<?> table, String instantTime) {
+  public MarkerFiles(HoodieTable table, String instantTime) {
     this(table.getMetaClient().getFs(),
         table.getMetaClient().getBasePath(),
         table.getMetaClient().getMarkerFolderPath(instantTime),
         instantTime);
   }
 
-  public void quietDeleteMarkerDir(JavaSparkContext jsc, int parallelism) {
+  public void quietDeleteMarkerDir(HoodieEngineContext context, int parallelism) {
     try {
-      deleteMarkerDir(jsc, parallelism);
+      deleteMarkerDir(context, parallelism);
     } catch (HoodieIOException ioe) {
       LOG.warn("Error deleting marker directory for instant " + instantTime, ioe);
     }
@@ -81,10 +81,10 @@
   /**
    * Delete Marker directory corresponding to an instant.
    *
-   * @param jsc Java Spark Context.
-   * @param parallelism Spark parallelism for deletion.
+   * @param context HoodieEngineContext.
+   * @param parallelism parallelism for deletion.
    */
-  public boolean deleteMarkerDir(JavaSparkContext jsc, int parallelism) {
+  public boolean deleteMarkerDir(HoodieEngineContext context, int parallelism) {
     try {
       if (fs.exists(markerDirPath)) {
         FileStatus[] fileStatuses = fs.listStatus(markerDirPath);
@@ -95,11 +95,11 @@
         if (markerDirSubPaths.size() > 0) {
           SerializableConfiguration conf = new SerializableConfiguration(fs.getConf());
           parallelism = Math.min(markerDirSubPaths.size(), parallelism);
-          jsc.parallelize(markerDirSubPaths, parallelism).foreach(subPathStr -> {
+          context.foreach(markerDirSubPaths, subPathStr -> {
             Path subPath = new Path(subPathStr);
             FileSystem fileSystem = subPath.getFileSystem(conf.get());
             fileSystem.delete(subPath, true);
-          });
+          }, parallelism);
         }
 
         boolean result = fs.delete(markerDirPath, true);
@@ -116,7 +116,7 @@
     return fs.exists(markerDirPath);
   }
 
-  public Set<String> createdAndMergedDataPaths(JavaSparkContext jsc, int parallelism) throws IOException {
+  public Set<String> createdAndMergedDataPaths(HoodieEngineContext context, int parallelism) throws IOException {
     Set<String> dataFiles = new HashSet<>();
 
     FileStatus[] topLevelStatuses = fs.listStatus(markerDirPath);
@@ -135,7 +135,7 @@
     if (subDirectories.size() > 0) {
       parallelism = Math.min(subDirectories.size(), parallelism);
       SerializableConfiguration serializedConf = new SerializableConfiguration(fs.getConf());
-      dataFiles.addAll(jsc.parallelize(subDirectories, parallelism).flatMap(directory -> {
+      dataFiles.addAll(context.flatMap(subDirectories, directory -> {
         Path path = new Path(directory);
         FileSystem fileSystem = path.getFileSystem(serializedConf.get());
         RemoteIterator<LocatedFileStatus> itr = fileSystem.listFiles(path, true);
@@ -147,8 +147,8 @@
             result.add(translateMarkerToDataPath(pathStr));
           }
         }
-        return result.iterator();
-      }).collect());
+        return result.stream();
+      }, parallelism));
     }
 
     return dataFiles;
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/WorkloadProfile.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/WorkloadProfile.java
new file mode 100644
index 0000000..a56710b
--- /dev/null
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/WorkloadProfile.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.table;
+
+import org.apache.hudi.common.util.collection.Pair;
+
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.Set;
+
+/**
+ * Information about incoming records for upsert/insert obtained either via sampling or introspecting the data fully.
+ * <p>
+ * TODO(vc): Think about obtaining this directly from index.tagLocation
+ */
+public class WorkloadProfile implements Serializable {
+
+  /**
+   * Computed workload profile.
+   */
+  protected final HashMap<String, WorkloadStat> partitionPathStatMap;
+
+  /**
+   * Global workloadStat.
+   */
+  protected final WorkloadStat globalStat;
+
+  public WorkloadProfile(Pair<HashMap<String, WorkloadStat>, WorkloadStat> profile) {
+    this.partitionPathStatMap = profile.getLeft();
+    this.globalStat = profile.getRight();
+  }
+
+  public WorkloadStat getGlobalStat() {
+    return globalStat;
+  }
+
+  public Set<String> getPartitionPaths() {
+    return partitionPathStatMap.keySet();
+  }
+
+  public HashMap<String, WorkloadStat> getPartitionPathStatMap() {
+    return partitionPathStatMap;
+  }
+
+  public WorkloadStat getWorkloadStat(String partitionPath) {
+    return partitionPathStatMap.get(partitionPath);
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder("WorkloadProfile {");
+    sb.append("globalStat=").append(globalStat).append(", ");
+    sb.append("partitionStat=").append(partitionPathStatMap);
+    sb.append('}');
+    return sb.toString();
+  }
+}
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/WorkloadStat.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/WorkloadStat.java
similarity index 94%
rename from hudi-client/src/main/java/org/apache/hudi/table/WorkloadStat.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/WorkloadStat.java
index f69e599..6fdb217 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/WorkloadStat.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/WorkloadStat.java
@@ -39,11 +39,11 @@
     updateLocationToCount = new HashMap<>();
   }
 
-  long addInserts(long numInserts) {
+  public long addInserts(long numInserts) {
     return this.numInserts += numInserts;
   }
 
-  long addUpdates(HoodieRecordLocation location, long numUpdates) {
+  public long addUpdates(HoodieRecordLocation location, long numUpdates) {
     updateLocationToCount.put(location.getFileId(), Pair.of(location.getInstantTime(), numUpdates));
     return this.numUpdates += numUpdates;
   }
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/BaseActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/BaseActionExecutor.java
similarity index 69%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/BaseActionExecutor.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/BaseActionExecutor.java
index 9ee4126..96189e7 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/BaseActionExecutor.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/BaseActionExecutor.java
@@ -21,24 +21,25 @@
 import java.io.Serializable;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.table.HoodieTable;
-import org.apache.spark.api.java.JavaSparkContext;
 
-public abstract class BaseActionExecutor<R> implements Serializable {
+public abstract class BaseActionExecutor<T extends HoodieRecordPayload, I, K, O, R> implements Serializable {
 
-  protected final transient JavaSparkContext jsc;
+  protected final transient HoodieEngineContext context;
   protected final transient Configuration hadoopConf;
 
   protected final HoodieWriteConfig config;
 
-  protected final HoodieTable<?> table;
+  protected final HoodieTable<T, I, K, O> table;
 
   protected final String instantTime;
 
-  public BaseActionExecutor(JavaSparkContext jsc, HoodieWriteConfig config, HoodieTable<?> table, String instantTime) {
-    this.jsc = jsc;
-    this.hadoopConf = jsc.hadoopConfiguration();
+  public BaseActionExecutor(HoodieEngineContext context, HoodieWriteConfig config, HoodieTable<T, I, K, O> table, String instantTime) {
+    this.context = context;
+    this.hadoopConf = context.getHadoopConf().get();
     this.config = config;
     this.table = table;
     this.instantTime = instantTime;
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/HoodieWriteMetadata.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/HoodieWriteMetadata.java
similarity index 91%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/HoodieWriteMetadata.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/HoodieWriteMetadata.java
index 03ddba3..5ef204f 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/HoodieWriteMetadata.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/HoodieWriteMetadata.java
@@ -18,11 +18,9 @@
 
 package org.apache.hudi.table.action;
 
-import org.apache.hudi.client.WriteStatus;
 import org.apache.hudi.common.model.HoodieCommitMetadata;
 import org.apache.hudi.common.model.HoodieWriteStat;
 import org.apache.hudi.common.util.Option;
-import org.apache.spark.api.java.JavaRDD;
 
 import java.time.Duration;
 import java.util.Collections;
@@ -32,9 +30,9 @@
 /**
  * Contains metadata, write-statuses and latency times corresponding to a commit/delta-commit action.
  */
-public class HoodieWriteMetadata {
+public class HoodieWriteMetadata<O> {
 
-  private JavaRDD<WriteStatus> writeStatuses;
+  private O writeStatuses;
   private Option<Duration> indexLookupDuration = Option.empty();
 
   // Will be set when auto-commit happens
@@ -48,7 +46,7 @@
   public HoodieWriteMetadata() {
   }
 
-  public JavaRDD<WriteStatus> getWriteStatuses() {
+  public O getWriteStatuses() {
     return writeStatuses;
   }
 
@@ -56,7 +54,7 @@
     return commitMetadata;
   }
 
-  public void setWriteStatuses(JavaRDD<WriteStatus> writeStatuses) {
+  public void setWriteStatuses(O writeStatuses) {
     this.writeStatuses = writeStatuses;
   }
 
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapRecordConsumer.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapRecordConsumer.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapRecordConsumer.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapRecordConsumer.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapUtils.java
similarity index 94%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapUtils.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapUtils.java
index cb4bc2d..ade2b18 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapUtils.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapUtils.java
@@ -26,13 +26,12 @@
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hudi.avro.model.HoodieFileStatus;
+import org.apache.hudi.client.common.HoodieEngineContext;
 import org.apache.hudi.common.bootstrap.FileStatusUtils;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
 import org.apache.hudi.common.util.collection.Pair;
 
-import org.apache.spark.api.java.JavaSparkContext;
-
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -47,12 +46,12 @@
    * Returns leaf folders with files under a path.
    * @param metaClient Hoodie table metadata client
    * @param fs  File System
-   * @param jsc Java spark context
+   * @param context JHoodieEngineContext
    * @return list of partition paths with files under them.
    * @throws IOException
    */
   public static List<Pair<String, List<HoodieFileStatus>>> getAllLeafFoldersWithFiles(HoodieTableMetaClient metaClient,
-      FileSystem fs, String basePathStr, JavaSparkContext jsc) throws IOException {
+      FileSystem fs, String basePathStr, HoodieEngineContext context) throws IOException {
     final Path basePath = new Path(basePathStr);
     final String baseFileExtension = metaClient.getTableConfig().getBaseFileFormat().getFileExtension();
     final Map<Integer, List<String>> levelToPartitions = new HashMap<>();
@@ -77,7 +76,7 @@
     }
 
     if (subDirectories.size() > 0) {
-      result.addAll(jsc.parallelize(subDirectories, subDirectories.size()).flatMap(directory -> {
+      result.addAll(context.flatMap(subDirectories, directory -> {
         PathFilter pathFilter = getFilePathFilter(baseFileExtension);
         Path path = new Path(directory);
         FileSystem fileSystem = path.getFileSystem(new Configuration());
@@ -92,8 +91,8 @@
             res.add(Pair.of(hoodieFileStatus, Pair.of(level, relativePath)));
           }
         }
-        return res.iterator();
-      }).collect());
+        return res.stream();
+      }, subDirectories.size()));
     }
 
     result.forEach(val -> {
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/bootstrap/HoodieBootstrapWriteMetadata.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/bootstrap/HoodieBootstrapWriteMetadata.java
similarity index 77%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/bootstrap/HoodieBootstrapWriteMetadata.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/bootstrap/HoodieBootstrapWriteMetadata.java
index 4e6167e..0639876 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/bootstrap/HoodieBootstrapWriteMetadata.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/bootstrap/HoodieBootstrapWriteMetadata.java
@@ -24,22 +24,22 @@
 /**
  * Write Result for a bootstrap operation.
  */
-public class HoodieBootstrapWriteMetadata {
+public class HoodieBootstrapWriteMetadata<O> {
 
-  private final Option<HoodieWriteMetadata> metadataBootstrapResult;
-  private final Option<HoodieWriteMetadata> fullBootstrapResult;
+  private final Option<HoodieWriteMetadata<O>> metadataBootstrapResult;
+  private final Option<HoodieWriteMetadata<O>> fullBootstrapResult;
 
-  public HoodieBootstrapWriteMetadata(Option<HoodieWriteMetadata> metadataBootstrapResult,
-                                      Option<HoodieWriteMetadata> fullBootstrapResult) {
+  public HoodieBootstrapWriteMetadata(Option<HoodieWriteMetadata<O>> metadataBootstrapResult,
+                                      Option<HoodieWriteMetadata<O>> fullBootstrapResult) {
     this.metadataBootstrapResult = metadataBootstrapResult;
     this.fullBootstrapResult = fullBootstrapResult;
   }
 
-  public Option<HoodieWriteMetadata> getMetadataBootstrapResult() {
+  public Option<HoodieWriteMetadata<O>> getMetadataBootstrapResult() {
     return metadataBootstrapResult;
   }
 
-  public Option<HoodieWriteMetadata> getFullBootstrapResult() {
+  public Option<HoodieWriteMetadata<O>> getFullBootstrapResult() {
     return fullBootstrapResult;
   }
 }
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/BaseCleanActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/BaseCleanActionExecutor.java
new file mode 100644
index 0000000..588437c
--- /dev/null
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/BaseCleanActionExecutor.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.hudi.table.action.clean;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.avro.model.HoodieActionInstant;
+import org.apache.hudi.avro.model.HoodieCleanMetadata;
+import org.apache.hudi.avro.model.HoodieCleanerPlan;
+import org.apache.hudi.avro.model.HoodieCleanFileInfo;
+import org.apache.hudi.common.HoodieCleanStat;
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.common.model.HoodieCleaningPolicy;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
+import org.apache.hudi.common.util.CleanerUtils;
+import org.apache.hudi.common.util.CollectionUtils;
+import org.apache.hudi.common.util.HoodieTimer;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.table.action.BaseActionExecutor;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+public abstract class BaseCleanActionExecutor<T extends HoodieRecordPayload, I, K, O> extends BaseActionExecutor<T, I, K, O, HoodieCleanMetadata> {
+
+  private static final long serialVersionUID = 1L;
+  private static final Logger LOG = LogManager.getLogger(BaseCleanActionExecutor.class);
+
+  public BaseCleanActionExecutor(HoodieEngineContext context, HoodieWriteConfig config, HoodieTable<T, I, K, O> table, String instantTime) {
+    super(context, config, table, instantTime);
+  }
+
+  /**
+   * Generates List of files to be cleaned.
+   *
+   * @param context HoodieEngineContext
+   * @return Cleaner Plan
+   */
+  HoodieCleanerPlan requestClean(HoodieEngineContext context) {
+    try {
+      CleanPlanner<T, I, K, O> planner = new CleanPlanner<>(table, config);
+      Option<HoodieInstant> earliestInstant = planner.getEarliestCommitToRetain();
+      List<String> partitionsToClean = planner.getPartitionPathsToClean(earliestInstant);
+
+      if (partitionsToClean.isEmpty()) {
+        LOG.info("Nothing to clean here. It is already clean");
+        return HoodieCleanerPlan.newBuilder().setPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS.name()).build();
+      }
+      LOG.info("Total Partitions to clean : " + partitionsToClean.size() + ", with policy " + config.getCleanerPolicy());
+      int cleanerParallelism = Math.min(partitionsToClean.size(), config.getCleanerParallelism());
+      LOG.info("Using cleanerParallelism: " + cleanerParallelism);
+
+      context.setJobStatus(this.getClass().getSimpleName(), "Generates list of file slices to be cleaned");
+
+      Map<String, List<HoodieCleanFileInfo>> cleanOps = context
+          .map(partitionsToClean, partitionPathToClean -> Pair.of(partitionPathToClean, planner.getDeletePaths(partitionPathToClean)), cleanerParallelism)
+          .stream()
+          .collect(Collectors.toMap(Pair::getKey, y -> CleanerUtils.convertToHoodieCleanFileInfoList(y.getValue())));
+
+      return new HoodieCleanerPlan(earliestInstant
+          .map(x -> new HoodieActionInstant(x.getTimestamp(), x.getAction(), x.getState().name())).orElse(null),
+          config.getCleanerPolicy().name(), CollectionUtils.createImmutableMap(),
+          CleanPlanner.LATEST_CLEAN_PLAN_VERSION, cleanOps);
+    } catch (IOException e) {
+      throw new HoodieIOException("Failed to schedule clean operation", e);
+    }
+  }
+
+  protected static Boolean deleteFileAndGetResult(FileSystem fs, String deletePathStr) throws IOException {
+    Path deletePath = new Path(deletePathStr);
+    LOG.debug("Working on delete path :" + deletePath);
+    try {
+      boolean deleteResult = fs.delete(deletePath, false);
+      if (deleteResult) {
+        LOG.debug("Cleaned file at path :" + deletePath);
+      }
+      return deleteResult;
+    } catch (FileNotFoundException fio) {
+      // With cleanPlan being used for retried cleaning operations, its possible to clean a file twice
+      return false;
+    }
+  }
+
+  /**
+   * Performs cleaning of partition paths according to cleaning policy and returns the number of files cleaned. Handles
+   * skews in partitions to clean by making files to clean as the unit of task distribution.
+   *
+   * @throws IllegalArgumentException if unknown cleaning policy is provided
+   */
+  abstract List<HoodieCleanStat> clean(HoodieEngineContext context, HoodieCleanerPlan cleanerPlan);
+
+  /**
+   * Creates a Cleaner plan if there are files to be cleaned and stores them in instant file.
+   * Cleaner Plan contains absolute file paths.
+   *
+   * @param startCleanTime Cleaner Instant Time
+   * @return Cleaner Plan if generated
+   */
+  Option<HoodieCleanerPlan> requestClean(String startCleanTime) {
+    final HoodieCleanerPlan cleanerPlan = requestClean(context);
+    if ((cleanerPlan.getFilePathsToBeDeletedPerPartition() != null)
+        && !cleanerPlan.getFilePathsToBeDeletedPerPartition().isEmpty()
+        && cleanerPlan.getFilePathsToBeDeletedPerPartition().values().stream().mapToInt(List::size).sum() > 0) {
+      // Only create cleaner plan which does some work
+      final HoodieInstant cleanInstant = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.CLEAN_ACTION, startCleanTime);
+      // Save to both aux and timeline folder
+      try {
+        table.getActiveTimeline().saveToCleanRequested(cleanInstant, TimelineMetadataUtils.serializeCleanerPlan(cleanerPlan));
+        LOG.info("Requesting Cleaning with instant time " + cleanInstant);
+      } catch (IOException e) {
+        LOG.error("Got exception when saving cleaner requested file", e);
+        throw new HoodieIOException(e.getMessage(), e);
+      }
+      return Option.of(cleanerPlan);
+    }
+    return Option.empty();
+  }
+
+  /**
+   * Executes the Cleaner plan stored in the instant metadata.
+   */
+  void runPendingClean(HoodieTable<T, I, K, O> table, HoodieInstant cleanInstant) {
+    try {
+      HoodieCleanerPlan cleanerPlan = CleanerUtils.getCleanerPlan(table.getMetaClient(), cleanInstant);
+      runClean(table, cleanInstant, cleanerPlan);
+    } catch (IOException e) {
+      throw new HoodieIOException(e.getMessage(), e);
+    }
+  }
+
+  private HoodieCleanMetadata runClean(HoodieTable<T, I, K, O> table, HoodieInstant cleanInstant, HoodieCleanerPlan cleanerPlan) {
+    ValidationUtils.checkArgument(cleanInstant.getState().equals(HoodieInstant.State.REQUESTED)
+        || cleanInstant.getState().equals(HoodieInstant.State.INFLIGHT));
+
+    try {
+      final HoodieInstant inflightInstant;
+      final HoodieTimer timer = new HoodieTimer();
+      timer.startTimer();
+      if (cleanInstant.isRequested()) {
+        inflightInstant = table.getActiveTimeline().transitionCleanRequestedToInflight(cleanInstant,
+            TimelineMetadataUtils.serializeCleanerPlan(cleanerPlan));
+      } else {
+        inflightInstant = cleanInstant;
+      }
+
+      List<HoodieCleanStat> cleanStats = clean(context, cleanerPlan);
+      if (cleanStats.isEmpty()) {
+        return HoodieCleanMetadata.newBuilder().build();
+      }
+
+      table.getMetaClient().reloadActiveTimeline();
+      HoodieCleanMetadata metadata = CleanerUtils.convertCleanMetadata(
+          inflightInstant.getTimestamp(),
+          Option.of(timer.endTimer()),
+          cleanStats
+      );
+
+      table.getActiveTimeline().transitionCleanInflightToComplete(inflightInstant,
+          TimelineMetadataUtils.serializeCleanMetadata(metadata));
+      LOG.info("Marked clean started on " + inflightInstant.getTimestamp() + " as complete");
+      return metadata;
+    } catch (IOException e) {
+      throw new HoodieIOException("Failed to clean up after commit", e);
+    }
+  }
+
+  @Override
+  public HoodieCleanMetadata execute() {
+    // If there are inflight(failed) or previously requested clean operation, first perform them
+    List<HoodieInstant> pendingCleanInstants = table.getCleanTimeline()
+        .filterInflightsAndRequested().getInstants().collect(Collectors.toList());
+    if (pendingCleanInstants.size() > 0) {
+      pendingCleanInstants.forEach(hoodieInstant -> {
+        LOG.info("Finishing previously unfinished cleaner instant=" + hoodieInstant);
+        try {
+          runPendingClean(table, hoodieInstant);
+        } catch (Exception e) {
+          LOG.warn("Failed to perform previous clean operation, instant: " + hoodieInstant, e);
+        }
+      });
+      table.getMetaClient().reloadActiveTimeline();
+    }
+
+    // Plan and execute a new clean action
+    Option<HoodieCleanerPlan> cleanerPlanOpt = requestClean(instantTime);
+    if (cleanerPlanOpt.isPresent()) {
+      table.getMetaClient().reloadActiveTimeline();
+      HoodieCleanerPlan cleanerPlan = cleanerPlanOpt.get();
+      if ((cleanerPlan.getFilePathsToBeDeletedPerPartition() != null) && !cleanerPlan.getFilePathsToBeDeletedPerPartition().isEmpty()) {
+        return runClean(table, HoodieTimeline.getCleanRequestedInstant(instantTime), cleanerPlan);
+      }
+    }
+    return null;
+  }
+}
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java
similarity index 98%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java
index dc89126..405fc81 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanner.java
@@ -63,7 +63,7 @@
  * <p>
  * 2) It bounds the growth of the files in the file system
  */
-public class CleanPlanner<T extends HoodieRecordPayload<T>> implements Serializable {
+public class CleanPlanner<T extends HoodieRecordPayload, I, K, O> implements Serializable {
 
   private static final Logger LOG = LogManager.getLogger(CleanPlanner.class);
 
@@ -74,10 +74,10 @@
   private final SyncableFileSystemView fileSystemView;
   private final HoodieTimeline commitTimeline;
   private final Map<HoodieFileGroupId, CompactionOperation> fgIdToPendingCompactionOperations;
-  private HoodieTable<T> hoodieTable;
+  private HoodieTable<T, I, K, O> hoodieTable;
   private HoodieWriteConfig config;
 
-  public CleanPlanner(HoodieTable<T> hoodieTable, HoodieWriteConfig config) {
+  public CleanPlanner(HoodieTable<T, I, K, O> hoodieTable, HoodieWriteConfig config) {
     this.hoodieTable = hoodieTable;
     this.fileSystemView = hoodieTable.getHoodieView();
     this.commitTimeline = hoodieTable.getCompletedCommitTimeline();
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/clean/PartitionCleanStat.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/PartitionCleanStat.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/clean/PartitionCleanStat.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/PartitionCleanStat.java
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractBulkInsertHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractBulkInsertHelper.java
new file mode 100644
index 0000000..3ead7a0
--- /dev/null
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractBulkInsertHelper.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.hudi.table.action.commit;
+
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.table.BulkInsertPartitioner;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.table.action.HoodieWriteMetadata;
+
+public abstract class AbstractBulkInsertHelper<T extends HoodieRecordPayload, I, K, O, R> {
+
+  public abstract HoodieWriteMetadata<O> bulkInsert(I inputRecords, String instantTime,
+                                                    HoodieTable<T, I, K, O> table, HoodieWriteConfig config,
+                                                    BaseCommitActionExecutor<T, I, K, O, R> executor, boolean performDedupe,
+                                                    Option<BulkInsertPartitioner<T>> userDefinedBulkInsertPartitioner);
+}
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractDeleteHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractDeleteHelper.java
new file mode 100644
index 0000000..c2d2df7
--- /dev/null
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractDeleteHelper.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.hudi.table.action.commit;
+
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.table.action.HoodieWriteMetadata;
+
+/**
+ * Helper class to perform delete keys on hoodie table.
+ *
+ * @param <T>
+ */
+public abstract class AbstractDeleteHelper<T extends HoodieRecordPayload, I, K, O, R> {
+
+  /**
+   * Deduplicate Hoodie records, using the given deduplication function.
+   *
+   * @param keys HoodieKeys to deduplicate
+   * @param table target Hoodie table for deduplicating
+   * @param parallelism parallelism or partitions to be used while reducing/deduplicating
+   * @return HoodieKey already be deduplicated
+   */
+  public abstract K deduplicateKeys(K keys, HoodieTable<T, I, K, O> table, int parallelism);
+
+  public abstract HoodieWriteMetadata<O> execute(String instantTime,
+                                                 K keys, HoodieEngineContext context,
+                                                 HoodieWriteConfig config, HoodieTable<T, I, K, O> table,
+                                                 BaseCommitActionExecutor<T, I, K, O, R> deleteExecutor);
+}
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/MergeHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractMergeHelper.java
similarity index 62%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/commit/MergeHelper.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractMergeHelper.java
index 4daa5c6..1bbffad 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/MergeHelper.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractMergeHelper.java
@@ -30,10 +30,8 @@
 import org.apache.hudi.client.utils.MergingIterator;
 import org.apache.hudi.common.model.HoodieBaseFile;
 import org.apache.hudi.common.model.HoodieRecordPayload;
-import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor;
 import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer;
 import org.apache.hudi.exception.HoodieException;
-import org.apache.hudi.execution.SparkBoundedInMemoryExecutor;
 import org.apache.hudi.io.HoodieMergeHandle;
 import org.apache.hudi.io.storage.HoodieFileReader;
 import org.apache.hudi.io.storage.HoodieFileReaderFactory;
@@ -49,67 +47,17 @@
 /**
  * Helper to read records from previous version of parquet and run Merge.
  */
-public class MergeHelper {
+public abstract class AbstractMergeHelper<T extends HoodieRecordPayload, I, K, O> {
 
   /**
    * Read records from previous version of base file and merge.
    * @param table Hoodie Table
    * @param upsertHandle Merge Handle
-   * @param <T>
    * @throws IOException in case of error
    */
-  public static <T extends HoodieRecordPayload<T>> void runMerge(HoodieTable<T> table, HoodieMergeHandle<T> upsertHandle) throws IOException {
-    final boolean externalSchemaTransformation = table.getConfig().shouldUseExternalSchemaTransformation();
-    Configuration cfgForHoodieFile = new Configuration(table.getHadoopConf());
-    HoodieBaseFile baseFile = upsertHandle.baseFileForMerge();
+  public abstract void runMerge(HoodieTable<T, I, K, O> table, HoodieMergeHandle<T, I, K, O> upsertHandle) throws IOException;
 
-    final GenericDatumWriter<GenericRecord> gWriter;
-    final GenericDatumReader<GenericRecord> gReader;
-    Schema readSchema;
-    if (externalSchemaTransformation || baseFile.getBootstrapBaseFile().isPresent()) {
-      readSchema = HoodieFileReaderFactory.getFileReader(table.getHadoopConf(), upsertHandle.getOldFilePath()).getSchema();
-      gWriter = new GenericDatumWriter<>(readSchema);
-      gReader = new GenericDatumReader<>(readSchema, upsertHandle.getWriterSchemaWithMetafields());
-    } else {
-      gReader = null;
-      gWriter = null;
-      readSchema = upsertHandle.getWriterSchemaWithMetafields();
-    }
-
-    BoundedInMemoryExecutor<GenericRecord, GenericRecord, Void> wrapper = null;
-    HoodieFileReader<GenericRecord> reader = HoodieFileReaderFactory.<T, GenericRecord>getFileReader(cfgForHoodieFile, upsertHandle.getOldFilePath());
-    try {
-      final Iterator<GenericRecord> readerIterator;
-      if (baseFile.getBootstrapBaseFile().isPresent()) {
-        readerIterator = getMergingIterator(table, upsertHandle, baseFile, reader, readSchema, externalSchemaTransformation);
-      } else {
-        readerIterator = reader.getRecordIterator(readSchema);
-      }
-
-      ThreadLocal<BinaryEncoder> encoderCache = new ThreadLocal<>();
-      ThreadLocal<BinaryDecoder> decoderCache = new ThreadLocal<>();
-      wrapper = new SparkBoundedInMemoryExecutor(table.getConfig(), readerIterator,
-          new UpdateHandler(upsertHandle), record -> {
-        if (!externalSchemaTransformation) {
-          return record;
-        }
-        return transformRecordBasedOnNewSchema(gReader, gWriter, encoderCache, decoderCache, (GenericRecord) record);
-      });
-      wrapper.execute();
-    } catch (Exception e) {
-      throw new HoodieException(e);
-    } finally {
-      if (reader != null) {
-        reader.close();
-      }
-      upsertHandle.close();
-      if (null != wrapper) {
-        wrapper.shutdownNow();
-      }
-    }
-  }
-
-  private static GenericRecord transformRecordBasedOnNewSchema(GenericDatumReader<GenericRecord> gReader, GenericDatumWriter<GenericRecord> gWriter,
+  protected GenericRecord transformRecordBasedOnNewSchema(GenericDatumReader<GenericRecord> gReader, GenericDatumWriter<GenericRecord> gWriter,
                                                                ThreadLocal<BinaryEncoder> encoderCache, ThreadLocal<BinaryDecoder> decoderCache,
                                                                GenericRecord gRec) {
     ByteArrayOutputStream inStream = null;
@@ -141,7 +89,7 @@
    * for indexing, writing and other functionality.
    *
    */
-  private static <T extends HoodieRecordPayload<T>> Iterator<GenericRecord> getMergingIterator(HoodieTable<T> table, HoodieMergeHandle<T> mergeHandle,
+  protected Iterator<GenericRecord> getMergingIterator(HoodieTable<T, I, K, O> table, HoodieMergeHandle<T, I, K, O> mergeHandle,
                                                                                                HoodieBaseFile baseFile, HoodieFileReader<GenericRecord> reader,
                                                                                                Schema readSchema, boolean externalSchemaTransformation) throws IOException {
     Path externalFilePath = new Path(baseFile.getBootstrapBaseFile().get().getPath());
@@ -161,11 +109,11 @@
   /**
    * Consumer that dequeues records from queue and sends to Merge Handle.
    */
-  private static class UpdateHandler extends BoundedInMemoryQueueConsumer<GenericRecord, Void> {
+  protected static class UpdateHandler extends BoundedInMemoryQueueConsumer<GenericRecord, Void> {
 
     private final HoodieMergeHandle upsertHandle;
 
-    private UpdateHandler(HoodieMergeHandle upsertHandle) {
+    protected UpdateHandler(HoodieMergeHandle upsertHandle) {
       this.upsertHandle = upsertHandle;
     }
 
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractWriteHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractWriteHelper.java
new file mode 100644
index 0000000..f5e5e35
--- /dev/null
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/AbstractWriteHelper.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.hudi.table.action.commit;
+
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.exception.HoodieUpsertException;
+import org.apache.hudi.index.HoodieIndex;
+import org.apache.hudi.table.HoodieTable;
+
+import org.apache.hudi.table.action.HoodieWriteMetadata;
+
+import java.time.Duration;
+import java.time.Instant;
+
+public abstract class AbstractWriteHelper<T extends HoodieRecordPayload, I, K, O, R> {
+
+  public HoodieWriteMetadata<O> write(String instantTime,
+                                      I inputRecordsRDD,
+                                      HoodieEngineContext context,
+                                      HoodieTable<T, I, K, O> table,
+                                      boolean shouldCombine,
+                                      int shuffleParallelism,
+                                      BaseCommitActionExecutor<T, I, K, O, R> executor,
+                                      boolean performTagging) {
+    try {
+      // De-dupe/merge if needed
+      I dedupedRecords =
+          combineOnCondition(shouldCombine, inputRecordsRDD, shuffleParallelism, table);
+
+      Instant lookupBegin = Instant.now();
+      I taggedRecords = dedupedRecords;
+      if (performTagging) {
+        // perform index loop up to get existing location of records
+        taggedRecords = tag(dedupedRecords, context, table);
+      }
+      Duration indexLookupDuration = Duration.between(lookupBegin, Instant.now());
+
+      HoodieWriteMetadata<O> result = executor.execute(taggedRecords);
+      result.setIndexLookupDuration(indexLookupDuration);
+      return result;
+    } catch (Throwable e) {
+      if (e instanceof HoodieUpsertException) {
+        throw (HoodieUpsertException) e;
+      }
+      throw new HoodieUpsertException("Failed to upsert for commit time " + instantTime, e);
+    }
+  }
+
+  private I tag(
+      I dedupedRecords, HoodieEngineContext context, HoodieTable<T, I, K, O> table) {
+    // perform index loop up to get existing location of records
+    return table.getIndex().tagLocation(dedupedRecords, context, table);
+  }
+
+  public I combineOnCondition(
+      boolean condition, I records, int parallelism, HoodieTable<T, I, K, O> table) {
+    return condition ? deduplicateRecords(records, table, parallelism) : records;
+  }
+
+  /**
+   * Deduplicate Hoodie records, using the given deduplication function.
+   *
+   * @param records     hoodieRecords to deduplicate
+   * @param parallelism parallelism or partitions to be used while reducing/deduplicating
+   * @return RDD of HoodieRecord already be deduplicated
+   */
+  public I deduplicateRecords(
+      I records, HoodieTable<T, I, K, O> table, int parallelism) {
+    return deduplicateRecords(records, table.getIndex(), parallelism);
+  }
+
+  public abstract I deduplicateRecords(
+      I records, HoodieIndex<T, I, K, O> index, int parallelism);
+}
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java
new file mode 100644
index 0000000..71de9b6
--- /dev/null
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.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.hudi.table.action.commit;
+
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.TaskContextSupplier;
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.common.model.HoodieCommitMetadata;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.model.HoodieWriteStat;
+import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieInstant.State;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieCommitException;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.table.WorkloadProfile;
+import org.apache.hudi.table.WorkloadStat;
+import org.apache.hudi.table.action.BaseActionExecutor;
+import org.apache.hudi.table.action.HoodieWriteMetadata;
+
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.time.Duration;
+import java.time.Instant;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+public abstract class BaseCommitActionExecutor<T extends HoodieRecordPayload, I, K, O, R>
+    extends BaseActionExecutor<T, I, K, O, R> {
+
+  private static final Logger LOG = LogManager.getLogger(BaseCommitActionExecutor.class);
+
+  protected final Option<Map<String, String>> extraMetadata;
+  protected final WriteOperationType operationType;
+  protected final TaskContextSupplier taskContextSupplier;
+
+  public BaseCommitActionExecutor(HoodieEngineContext context, HoodieWriteConfig config,
+                                  HoodieTable<T, I, K, O> table, String instantTime, WriteOperationType operationType,
+                                  Option<Map<String, String>> extraMetadata) {
+    super(context, config, table, instantTime);
+    this.operationType = operationType;
+    this.extraMetadata = extraMetadata;
+    this.taskContextSupplier = context.getTaskContextSupplier();
+  }
+
+  public abstract HoodieWriteMetadata<O> execute(I inputRecords);
+
+  /**
+   * Save the workload profile in an intermediate file (here re-using commit files) This is useful when performing
+   * rollback for MOR tables. Only updates are recorded in the workload profile metadata since updates to log blocks
+   * are unknown across batches Inserts (which are new parquet files) are rolled back based on commit time. // TODO :
+   * Create a new WorkloadProfile metadata file instead of using HoodieCommitMetadata
+   */
+  void saveWorkloadProfileMetadataToInflight(WorkloadProfile profile, String instantTime)
+      throws HoodieCommitException {
+    try {
+      HoodieCommitMetadata metadata = new HoodieCommitMetadata();
+      profile.getPartitionPaths().forEach(path -> {
+        WorkloadStat partitionStat = profile.getWorkloadStat(path.toString());
+        HoodieWriteStat insertStat = new HoodieWriteStat();
+        insertStat.setNumInserts(partitionStat.getNumInserts());
+        insertStat.setFileId("");
+        insertStat.setPrevCommit(HoodieWriteStat.NULL_COMMIT);
+        metadata.addWriteStat(path, insertStat);
+
+        partitionStat.getUpdateLocationToCount().forEach((key, value) -> {
+          HoodieWriteStat writeStat = new HoodieWriteStat();
+          writeStat.setFileId(key);
+          // TODO : Write baseCommitTime is possible here ?
+          writeStat.setPrevCommit(value.getKey());
+          writeStat.setNumUpdateWrites(value.getValue());
+          metadata.addWriteStat(path, writeStat);
+        });
+      });
+      metadata.setOperationType(operationType);
+
+      HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
+      String commitActionType = getCommitActionType();
+      HoodieInstant requested = new HoodieInstant(State.REQUESTED, commitActionType, instantTime);
+      activeTimeline.transitionRequestedToInflight(requested,
+          Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)),
+          config.shouldAllowMultiWriteOnSameInstant());
+    } catch (IOException io) {
+      throw new HoodieCommitException("Failed to commit " + instantTime + " unable to save inflight metadata ", io);
+    }
+  }
+
+  protected String getCommitActionType() {
+    return  table.getMetaClient().getCommitActionType();
+  }
+
+  protected void commitOnAutoCommit(HoodieWriteMetadata result) {
+    if (config.shouldAutoCommit()) {
+      LOG.info("Auto commit enabled: Committing " + instantTime);
+      commit(extraMetadata, result);
+    } else {
+      LOG.info("Auto commit disabled for " + instantTime);
+    }
+  }
+
+  protected abstract void commit(Option<Map<String, String>> extraMetadata, HoodieWriteMetadata<O> result);
+
+  /**
+   * Finalize Write operation.
+   * @param instantTime Instant Time
+   * @param stats Hoodie Write Stat
+   */
+  protected void finalizeWrite(String instantTime, List<HoodieWriteStat> stats, HoodieWriteMetadata result) {
+    try {
+      Instant start = Instant.now();
+      table.finalizeWrite(context, instantTime, stats);
+      result.setFinalizeDuration(Duration.between(start, Instant.now()));
+    } catch (HoodieIOException ioe) {
+      throw new HoodieCommitException("Failed to complete commit " + instantTime + " due to finalize errors.", ioe);
+    }
+  }
+
+  /**
+   * By default, return the writer schema in Write Config for storing in commit.
+   */
+  protected String getSchemaToStoreInCommit() {
+    return config.getSchema();
+  }
+
+  protected boolean isWorkloadProfileNeeded() {
+    return true;
+  }
+
+  protected abstract Iterator<List<WriteStatus>> handleInsert(String idPfx,
+      Iterator<HoodieRecord<T>> recordItr) throws Exception;
+
+  protected abstract Iterator<List<WriteStatus>> handleUpdate(String partitionPath, String fileId,
+      Iterator<HoodieRecord<T>> recordItr) throws IOException;
+}
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/BucketInfo.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BucketInfo.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/commit/BucketInfo.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BucketInfo.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/BucketType.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BucketType.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/commit/BucketType.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BucketType.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/InsertBucket.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/InsertBucket.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/commit/InsertBucket.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/InsertBucket.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/InsertBucketCumulativeWeightPair.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/InsertBucketCumulativeWeightPair.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/commit/InsertBucketCumulativeWeightPair.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/InsertBucketCumulativeWeightPair.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/SmallFile.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/SmallFile.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/commit/SmallFile.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/SmallFile.java
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/AbstractCompactHelpers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/AbstractCompactHelpers.java
new file mode 100644
index 0000000..3ff9e62
--- /dev/null
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/AbstractCompactHelpers.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.hudi.table.action.compact;
+
+import org.apache.hudi.common.model.HoodieCommitMetadata;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieCompactionException;
+import org.apache.hudi.table.HoodieTable;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+
+/**
+ * Base class helps to perform compact.
+ *
+ * @param <T> Type of payload in {@link org.apache.hudi.common.model.HoodieRecord}
+ * @param <I> Type of inputs
+ * @param <K> Type of keys
+ * @param <O> Type of outputs
+ */
+public abstract class AbstractCompactHelpers<T extends HoodieRecordPayload, I, K, O> {
+  public abstract HoodieCommitMetadata createCompactionMetadata(HoodieTable<T, I, K, O> table,
+                                                                String compactionInstantTime,
+                                                                O writeStatuses,
+                                                                String schema) throws IOException;
+
+  public void completeInflightCompaction(HoodieTable table, String compactionCommitTime, HoodieCommitMetadata commitMetadata) {
+    HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
+    try {
+      activeTimeline.transitionCompactionInflightToComplete(
+          new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionCommitTime),
+          Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
+    } catch (IOException e) {
+      throw new HoodieCompactionException(
+          "Failed to commit " + table.getMetaClient().getBasePath() + " at time " + compactionCommitTime, e);
+    }
+  }
+}
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/BaseScheduleCompactionActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/BaseScheduleCompactionActionExecutor.java
new file mode 100644
index 0000000..d21c7d9
--- /dev/null
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/BaseScheduleCompactionActionExecutor.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.table.action.compact;
+
+import org.apache.hudi.avro.model.HoodieCompactionPlan;
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.table.action.BaseActionExecutor;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+public abstract class BaseScheduleCompactionActionExecutor<T extends HoodieRecordPayload, I, K, O> extends BaseActionExecutor<T, I, K, O, Option<HoodieCompactionPlan>> {
+
+  private final Option<Map<String, String>> extraMetadata;
+
+  public BaseScheduleCompactionActionExecutor(HoodieEngineContext context,
+                                              HoodieWriteConfig config,
+                                              HoodieTable<T, I, K, O> table,
+                                              String instantTime,
+                                              Option<Map<String, String>> extraMetadata) {
+    super(context, config, table, instantTime);
+    this.extraMetadata = extraMetadata;
+  }
+
+  protected abstract HoodieCompactionPlan scheduleCompaction();
+
+  @Override
+  public Option<HoodieCompactionPlan> execute() {
+    // if there are inflight writes, their instantTime must not be less than that of compaction instant time
+    table.getActiveTimeline().getCommitsTimeline().filterPendingExcludingCompaction().firstInstant()
+        .ifPresent(earliestInflight -> ValidationUtils.checkArgument(
+            HoodieTimeline.compareTimestamps(earliestInflight.getTimestamp(), HoodieTimeline.GREATER_THAN, instantTime),
+            "Earliest write inflight instant time must be later than compaction time. Earliest :" + earliestInflight
+                + ", Compaction scheduled at " + instantTime));
+
+    // Committed and pending compaction instants should have strictly lower timestamps
+    List<HoodieInstant> conflictingInstants = table.getActiveTimeline()
+        .getCommitsAndCompactionTimeline().getInstants()
+        .filter(instant -> HoodieTimeline.compareTimestamps(
+            instant.getTimestamp(), HoodieTimeline.GREATER_THAN_OR_EQUALS, instantTime))
+        .collect(Collectors.toList());
+    ValidationUtils.checkArgument(conflictingInstants.isEmpty(),
+        "Following instants have timestamps >= compactionInstant (" + instantTime + ") Instants :"
+            + conflictingInstants);
+
+    HoodieCompactionPlan plan = scheduleCompaction();
+    if (plan != null && (plan.getOperations() != null) && (!plan.getOperations().isEmpty())) {
+      extraMetadata.ifPresent(plan::setExtraMetadata);
+      HoodieInstant compactionInstant =
+          new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, instantTime);
+      try {
+        table.getActiveTimeline().saveToCompactionRequested(compactionInstant,
+            TimelineMetadataUtils.serializeCompactionPlan(plan));
+      } catch (IOException ioe) {
+        throw new HoodieIOException("Exception scheduling compaction", ioe);
+      }
+      return Option.of(plan);
+    }
+    return Option.empty();
+  }
+}
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java
similarity index 72%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java
index c81b028..e5212e8 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java
@@ -19,14 +19,12 @@
 package org.apache.hudi.table.action.compact;
 
 import org.apache.hudi.avro.model.HoodieCompactionPlan;
-import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieEngineContext;
 import org.apache.hudi.common.model.HoodieFileGroupId;
+import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.table.HoodieTable;
 
-import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
-
 import java.io.IOException;
 import java.io.Serializable;
 import java.util.Set;
@@ -34,12 +32,12 @@
 /**
  * A HoodieCompactor runs compaction on a hoodie table.
  */
-public interface HoodieCompactor extends Serializable {
+public interface HoodieCompactor<T extends HoodieRecordPayload, I, K, O> extends Serializable {
 
   /**
    * Generate a new compaction plan for scheduling.
    *
-   * @param jsc Spark Context
+   * @param context HoodieEngineContext
    * @param hoodieTable Hoodie Table
    * @param config Hoodie Write Configuration
    * @param compactionCommitTime scheduled compaction commit time
@@ -47,12 +45,12 @@
    * @return Compaction Plan
    * @throws IOException when encountering errors
    */
-  HoodieCompactionPlan generateCompactionPlan(JavaSparkContext jsc, HoodieTable hoodieTable, HoodieWriteConfig config,
-      String compactionCommitTime, Set<HoodieFileGroupId> fgIdsInPendingCompactions) throws IOException;
+  HoodieCompactionPlan generateCompactionPlan(HoodieEngineContext context, HoodieTable<T, I, K, O> hoodieTable, HoodieWriteConfig config,
+                                              String compactionCommitTime, Set<HoodieFileGroupId> fgIdsInPendingCompactions) throws IOException;
 
   /**
    * Execute compaction operations and report back status.
    */
-  JavaRDD<WriteStatus> compact(JavaSparkContext jsc, HoodieCompactionPlan compactionPlan, HoodieTable hoodieTable,
+  O compact(HoodieEngineContext context, HoodieCompactionPlan compactionPlan, HoodieTable<T, I, K, O> hoodieTable,
       HoodieWriteConfig config, String compactionInstantTime) throws IOException;
 }
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/compact/OperationResult.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/OperationResult.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/compact/OperationResult.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/OperationResult.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/compact/strategy/BoundedIOCompactionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/BoundedIOCompactionStrategy.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/compact/strategy/BoundedIOCompactionStrategy.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/BoundedIOCompactionStrategy.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/compact/strategy/BoundedPartitionAwareCompactionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/BoundedPartitionAwareCompactionStrategy.java
similarity index 87%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/compact/strategy/BoundedPartitionAwareCompactionStrategy.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/BoundedPartitionAwareCompactionStrategy.java
index 597348f..747e0b2 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/compact/strategy/BoundedPartitionAwareCompactionStrategy.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/BoundedPartitionAwareCompactionStrategy.java
@@ -39,7 +39,7 @@
  */
 public class BoundedPartitionAwareCompactionStrategy extends DayBasedCompactionStrategy {
 
-  SimpleDateFormat dateFormat = new SimpleDateFormat(DATE_PARTITION_FORMAT);
+  SimpleDateFormat dateFormat = new SimpleDateFormat(DayBasedCompactionStrategy.DATE_PARTITION_FORMAT);
 
   @Override
   public List<HoodieCompactionOperation> orderAndFilter(HoodieWriteConfig writeConfig,
@@ -50,8 +50,8 @@
     // Filter out all partitions greater than earliestPartitionPathToCompact
 
     return operations.stream().collect(Collectors.groupingBy(HoodieCompactionOperation::getPartitionPath)).entrySet()
-        .stream().sorted(Map.Entry.comparingByKey(comparator))
-        .filter(e -> comparator.compare(earliestPartitionPathToCompact, e.getKey()) >= 0)
+        .stream().sorted(Map.Entry.comparingByKey(DayBasedCompactionStrategy.comparator))
+        .filter(e -> DayBasedCompactionStrategy.comparator.compare(earliestPartitionPathToCompact, e.getKey()) >= 0)
         .flatMap(e -> e.getValue().stream()).collect(Collectors.toList());
   }
 
@@ -63,7 +63,7 @@
     // Get all partitions and sort them
     return partitionPaths.stream().map(partition -> partition.replace("/", "-"))
         .sorted(Comparator.reverseOrder()).map(partitionPath -> partitionPath.replace("-", "/"))
-        .filter(e -> comparator.compare(earliestPartitionPathToCompact, e) >= 0).collect(Collectors.toList());
+        .filter(e -> DayBasedCompactionStrategy.comparator.compare(earliestPartitionPathToCompact, e) >= 0).collect(Collectors.toList());
   }
 
   public static Date getDateAtOffsetFromToday(int offset) {
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/compact/strategy/CompactionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/CompactionStrategy.java
similarity index 97%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/compact/strategy/CompactionStrategy.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/CompactionStrategy.java
index 0e8e4c6..6c631c4 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/compact/strategy/CompactionStrategy.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/CompactionStrategy.java
@@ -26,7 +26,6 @@
 import org.apache.hudi.common.util.CompactionUtils;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.config.HoodieWriteConfig;
-import org.apache.hudi.table.action.compact.HoodieMergeOnReadTableCompactor;
 
 import java.io.Serializable;
 import java.util.HashMap;
@@ -38,8 +37,6 @@
  * implementations of this abstract class can capture the relevant metrics to order and filter the final list of
  * compaction operation to run in a single compaction. Implementation of CompactionStrategy cannot hold any state.
  * Difference instantiations can be passed in every time
- *
- * @see HoodieMergeOnReadTableCompactor
  */
 public abstract class CompactionStrategy implements Serializable {
 
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/compact/strategy/DayBasedCompactionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/DayBasedCompactionStrategy.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/compact/strategy/DayBasedCompactionStrategy.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/DayBasedCompactionStrategy.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/compact/strategy/LogFileSizeBasedCompactionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/LogFileSizeBasedCompactionStrategy.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/compact/strategy/LogFileSizeBasedCompactionStrategy.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/LogFileSizeBasedCompactionStrategy.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/compact/strategy/UnBoundedCompactionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/UnBoundedCompactionStrategy.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/compact/strategy/UnBoundedCompactionStrategy.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/UnBoundedCompactionStrategy.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/compact/strategy/UnBoundedPartitionAwareCompactionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/UnBoundedPartitionAwareCompactionStrategy.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/compact/strategy/UnBoundedPartitionAwareCompactionStrategy.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/strategy/UnBoundedPartitionAwareCompactionStrategy.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java
similarity index 91%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java
index 0b4e4c4..0bbc0a0 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java
@@ -20,7 +20,9 @@
 
 import org.apache.hudi.avro.model.HoodieRestoreMetadata;
 import org.apache.hudi.avro.model.HoodieRollbackMetadata;
+import org.apache.hudi.client.common.HoodieEngineContext;
 import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
 import org.apache.hudi.common.table.timeline.HoodieInstant;
 import org.apache.hudi.common.table.timeline.HoodieTimeline;
@@ -32,7 +34,6 @@
 import org.apache.hudi.table.action.BaseActionExecutor;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
-import org.apache.spark.api.java.JavaSparkContext;
 
 import java.io.IOException;
 import java.util.Collections;
@@ -41,18 +42,18 @@
 import java.util.Map;
 import java.util.stream.Collectors;
 
-public abstract class BaseRestoreActionExecutor extends BaseActionExecutor<HoodieRestoreMetadata> {
+public abstract class BaseRestoreActionExecutor<T extends HoodieRecordPayload, I, K, O> extends BaseActionExecutor<T, I, K, O, HoodieRestoreMetadata> {
 
   private static final Logger LOG = LogManager.getLogger(BaseRestoreActionExecutor.class);
 
   private final String restoreInstantTime;
 
-  public BaseRestoreActionExecutor(JavaSparkContext jsc,
+  public BaseRestoreActionExecutor(HoodieEngineContext context,
                                    HoodieWriteConfig config,
-                                   HoodieTable<?> table,
+                                   HoodieTable<T, I, K, O> table,
                                    String instantTime,
                                    String restoreInstantTime) {
-    super(jsc, config, table, instantTime);
+    super(context, config, table, instantTime);
     this.restoreInstantTime = restoreInstantTime;
   }
 
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/rollback/MarkerBasedRollbackStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/AbstractMarkerBasedRollbackStrategy.java
similarity index 63%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/rollback/MarkerBasedRollbackStrategy.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/AbstractMarkerBasedRollbackStrategy.java
index 2a137b4..40526b8 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/rollback/MarkerBasedRollbackStrategy.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/AbstractMarkerBasedRollbackStrategy.java
@@ -18,63 +18,57 @@
 
 package org.apache.hudi.table.action.rollback;
 
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.client.common.HoodieEngineContext;
 import org.apache.hudi.common.HoodieRollbackStat;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.HoodieLogFile;
-import org.apache.hudi.common.model.IOType;
+import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.table.log.HoodieLogFormat;
 import org.apache.hudi.common.table.log.block.HoodieCommandBlock;
 import org.apache.hudi.common.table.log.block.HoodieLogBlock;
 import org.apache.hudi.common.table.timeline.HoodieInstant;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.exception.HoodieIOException;
-import org.apache.hudi.exception.HoodieRollbackException;
 import org.apache.hudi.table.HoodieTable;
-import org.apache.hudi.table.MarkerFiles;
-
-import org.apache.hadoop.fs.Path;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
-import org.apache.spark.api.java.JavaSparkContext;
 
 import java.io.IOException;
 import java.util.Collections;
-import java.util.List;
 import java.util.Map;
 
-import scala.Tuple2;
-
 /**
  * Performs rollback using marker files generated during the write..
  */
-public class MarkerBasedRollbackStrategy implements BaseRollbackActionExecutor.RollbackStrategy {
+public abstract class AbstractMarkerBasedRollbackStrategy<T extends HoodieRecordPayload, I, K, O> implements BaseRollbackActionExecutor.RollbackStrategy {
 
-  private static final Logger LOG = LogManager.getLogger(MarkerBasedRollbackStrategy.class);
+  private static final Logger LOG = LogManager.getLogger(AbstractMarkerBasedRollbackStrategy.class);
 
-  private final HoodieTable<?> table;
+  protected final HoodieTable<T, I, K, O> table;
 
-  private final transient JavaSparkContext jsc;
+  protected final transient HoodieEngineContext context;
 
-  private final HoodieWriteConfig config;
+  protected final HoodieWriteConfig config;
 
   private final String basePath;
 
   private final String instantTime;
 
-  public MarkerBasedRollbackStrategy(HoodieTable<?> table, JavaSparkContext jsc, HoodieWriteConfig config, String instantTime) {
+  public AbstractMarkerBasedRollbackStrategy(HoodieTable<T, I, K, O> table, HoodieEngineContext context, HoodieWriteConfig config, String instantTime) {
     this.table = table;
-    this.jsc = jsc;
+    this.context = context;
     this.basePath = table.getMetaClient().getBasePath();
     this.config = config;
     this.instantTime = instantTime;
   }
 
-  private HoodieRollbackStat undoMerge(String mergedBaseFilePath) throws IOException {
+  protected HoodieRollbackStat undoMerge(String mergedBaseFilePath) throws IOException {
     LOG.info("Rolling back by deleting the merged base file:" + mergedBaseFilePath);
     return deleteBaseFile(mergedBaseFilePath);
   }
 
-  private HoodieRollbackStat undoCreate(String createdBaseFilePath) throws IOException {
+  protected HoodieRollbackStat undoCreate(String createdBaseFilePath) throws IOException {
     LOG.info("Rolling back by deleting the created base file:" + createdBaseFilePath);
     return deleteBaseFile(createdBaseFilePath);
   }
@@ -89,7 +83,7 @@
         .build();
   }
 
-  private HoodieRollbackStat undoAppend(String appendBaseFilePath, HoodieInstant instantToRollback) throws IOException, InterruptedException {
+  protected HoodieRollbackStat undoAppend(String appendBaseFilePath, HoodieInstant instantToRollback) throws IOException, InterruptedException {
     Path baseFilePathForAppend = new Path(basePath, appendBaseFilePath);
     String fileId = FSUtils.getFileIdFromFilePath(baseFilePathForAppend);
     String baseCommitTime = FSUtils.getCommitTime(baseFilePathForAppend.getName());
@@ -131,33 +125,4 @@
         .withRollbackBlockAppendResults(Collections.emptyMap())
         .build();
   }
-
-  @Override
-  public List<HoodieRollbackStat> execute(HoodieInstant instantToRollback) {
-    try {
-      MarkerFiles markerFiles = new MarkerFiles(table, instantToRollback.getTimestamp());
-      List<String> markerFilePaths = markerFiles.allMarkerFilePaths();
-      int parallelism = Math.max(Math.min(markerFilePaths.size(), config.getRollbackParallelism()), 1);
-      return jsc.parallelize(markerFilePaths, parallelism)
-          .map(markerFilePath -> {
-            String typeStr = markerFilePath.substring(markerFilePath.lastIndexOf(".") + 1);
-            IOType type = IOType.valueOf(typeStr);
-            switch (type) {
-              case MERGE:
-                return undoMerge(MarkerFiles.stripMarkerSuffix(markerFilePath));
-              case APPEND:
-                return undoAppend(MarkerFiles.stripMarkerSuffix(markerFilePath), instantToRollback);
-              case CREATE:
-                return undoCreate(MarkerFiles.stripMarkerSuffix(markerFilePath));
-              default:
-                throw new HoodieRollbackException("Unknown marker type, during rollback of " + instantToRollback);
-            }
-          })
-          .mapToPair(rollbackStat -> new Tuple2<>(rollbackStat.getPartitionPath(), rollbackStat))
-          .reduceByKey(RollbackUtils::mergeRollbackStat)
-          .map(Tuple2::_2).collect();
-    } catch (Exception e) {
-      throw new HoodieRollbackException("Error rolling back using marker files written for " + instantToRollback, e);
-    }
-  }
 }
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/rollback/CopyOnWriteRollbackActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseCopyOnWriteRollbackActionExecutor.java
similarity index 61%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/rollback/CopyOnWriteRollbackActionExecutor.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseCopyOnWriteRollbackActionExecutor.java
index b06dc05..c60cb7f 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/rollback/CopyOnWriteRollbackActionExecutor.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseCopyOnWriteRollbackActionExecutor.java
@@ -18,7 +18,9 @@
 
 package org.apache.hudi.table.action.rollback;
 
+import org.apache.hudi.client.common.HoodieEngineContext;
 import org.apache.hudi.common.HoodieRollbackStat;
+import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
 import org.apache.hudi.common.table.timeline.HoodieInstant;
 import org.apache.hudi.common.util.HoodieTimer;
@@ -27,33 +29,32 @@
 
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
-import org.apache.spark.api.java.JavaSparkContext;
 
 import java.util.ArrayList;
 import java.util.List;
 
-public class CopyOnWriteRollbackActionExecutor extends BaseRollbackActionExecutor {
+public abstract class BaseCopyOnWriteRollbackActionExecutor<T extends HoodieRecordPayload, I, K, O> extends BaseRollbackActionExecutor<T, I, K, O> {
 
-  private static final Logger LOG = LogManager.getLogger(CopyOnWriteRollbackActionExecutor.class);
+  private static final Logger LOG = LogManager.getLogger(BaseCopyOnWriteRollbackActionExecutor.class);
 
-  public CopyOnWriteRollbackActionExecutor(JavaSparkContext jsc,
-      HoodieWriteConfig config,
-      HoodieTable<?> table,
-      String instantTime,
-      HoodieInstant commitInstant,
-      boolean deleteInstants) {
-    super(jsc, config, table, instantTime, commitInstant, deleteInstants);
+  public BaseCopyOnWriteRollbackActionExecutor(HoodieEngineContext context,
+                                               HoodieWriteConfig config,
+                                               HoodieTable<T, I, K, O> table,
+                                               String instantTime,
+                                               HoodieInstant commitInstant,
+                                               boolean deleteInstants) {
+    super(context, config, table, instantTime, commitInstant, deleteInstants);
   }
 
-  public CopyOnWriteRollbackActionExecutor(JavaSparkContext jsc,
-      HoodieWriteConfig config,
-      HoodieTable<?> table,
-      String instantTime,
-      HoodieInstant commitInstant,
-      boolean deleteInstants,
-      boolean skipTimelinePublish,
-      boolean useMarkerBasedStrategy) {
-    super(jsc, config, table, instantTime, commitInstant, deleteInstants, skipTimelinePublish, useMarkerBasedStrategy);
+  public BaseCopyOnWriteRollbackActionExecutor(HoodieEngineContext context,
+                                               HoodieWriteConfig config,
+                                               HoodieTable<T, I, K, O> table,
+                                               String instantTime,
+                                               HoodieInstant commitInstant,
+                                               boolean deleteInstants,
+                                               boolean skipTimelinePublish,
+                                               boolean useMarkerBasedStrategy) {
+    super(context, config, table, instantTime, commitInstant, deleteInstants, skipTimelinePublish, useMarkerBasedStrategy);
   }
 
   @Override
@@ -87,11 +88,4 @@
     LOG.info("Time(in ms) taken to finish rollback " + rollbackTimer.endTimer());
     return stats;
   }
-
-  @Override
-  protected List<HoodieRollbackStat> executeRollbackUsingFileListing(HoodieInstant instantToRollback) {
-    List<ListingBasedRollbackRequest> rollbackRequests = RollbackUtils.generateRollbackRequestsByListingCOW(table.getMetaClient().getFs(), table.getMetaClient().getBasePath(),
-        config.shouldAssumeDatePartitioning());
-    return new ListingBasedRollbackHelper(table.getMetaClient(), config).performRollback(jsc, instantToRollback, rollbackRequests);
-  }
 }
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/rollback/MergeOnReadRollbackActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseMergeOnReadRollbackActionExecutor.java
similarity index 65%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/rollback/MergeOnReadRollbackActionExecutor.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseMergeOnReadRollbackActionExecutor.java
index 97e110d..0e747a5 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/rollback/MergeOnReadRollbackActionExecutor.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseMergeOnReadRollbackActionExecutor.java
@@ -18,43 +18,42 @@
 
 package org.apache.hudi.table.action.rollback;
 
+import org.apache.hudi.client.common.HoodieEngineContext;
 import org.apache.hudi.common.HoodieRollbackStat;
+import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.table.timeline.HoodieInstant;
 import org.apache.hudi.common.util.HoodieTimer;
 import org.apache.hudi.config.HoodieWriteConfig;
-import org.apache.hudi.exception.HoodieIOException;
 import org.apache.hudi.table.HoodieTable;
-
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
-import org.apache.spark.api.java.JavaSparkContext;
 
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
-public class MergeOnReadRollbackActionExecutor extends BaseRollbackActionExecutor {
+public abstract class BaseMergeOnReadRollbackActionExecutor<T extends HoodieRecordPayload, I, K, O> extends BaseRollbackActionExecutor<T, I, K, O> {
 
-  private static final Logger LOG = LogManager.getLogger(MergeOnReadRollbackActionExecutor.class);
+  private static final Logger LOG = LogManager.getLogger(BaseMergeOnReadRollbackActionExecutor.class);
 
-  public MergeOnReadRollbackActionExecutor(JavaSparkContext jsc,
-      HoodieWriteConfig config,
-      HoodieTable<?> table,
-      String instantTime,
-      HoodieInstant commitInstant,
-      boolean deleteInstants) {
-    super(jsc, config, table, instantTime, commitInstant, deleteInstants);
+  public BaseMergeOnReadRollbackActionExecutor(HoodieEngineContext context,
+                                               HoodieWriteConfig config,
+                                               HoodieTable<T, I, K, O> table,
+                                               String instantTime,
+                                               HoodieInstant commitInstant,
+                                               boolean deleteInstants) {
+    super(context, config, table, instantTime, commitInstant, deleteInstants);
   }
 
-  public MergeOnReadRollbackActionExecutor(JavaSparkContext jsc,
-      HoodieWriteConfig config,
-      HoodieTable<?> table,
-      String instantTime,
-      HoodieInstant commitInstant,
-      boolean deleteInstants,
-      boolean skipTimelinePublish,
-      boolean useMarkerBasedStrategy) {
-    super(jsc, config, table, instantTime, commitInstant, deleteInstants, skipTimelinePublish, useMarkerBasedStrategy);
+  public BaseMergeOnReadRollbackActionExecutor(HoodieEngineContext context,
+                                               HoodieWriteConfig config,
+                                               HoodieTable<T, I, K, O> table,
+                                               String instantTime,
+                                               HoodieInstant commitInstant,
+                                               boolean deleteInstants,
+                                               boolean skipTimelinePublish,
+                                               boolean useMarkerBasedStrategy) {
+    super(context, config, table, instantTime, commitInstant, deleteInstants, skipTimelinePublish, useMarkerBasedStrategy);
   }
 
   @Override
@@ -95,15 +94,4 @@
     LOG.info("Time(in ms) taken to finish rollback " + rollbackTimer.endTimer());
     return allRollbackStats;
   }
-
-  @Override
-  protected List<HoodieRollbackStat> executeRollbackUsingFileListing(HoodieInstant resolvedInstant) {
-    List<ListingBasedRollbackRequest> rollbackRequests;
-    try {
-      rollbackRequests = RollbackUtils.generateRollbackRequestsUsingFileListingMOR(resolvedInstant, table, jsc);
-    } catch (IOException e) {
-      throw new HoodieIOException("Error generating rollback requests by file listing.", e);
-    }
-    return new ListingBasedRollbackHelper(table.getMetaClient(), config).performRollback(jsc, resolvedInstant, rollbackRequests);
-  }
 }
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java
similarity index 92%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java
index 36199c5..40206ca 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java
@@ -19,9 +19,11 @@
 package org.apache.hudi.table.action.rollback;
 
 import org.apache.hudi.avro.model.HoodieRollbackMetadata;
+import org.apache.hudi.client.common.HoodieEngineContext;
 import org.apache.hudi.common.HoodieRollbackStat;
 import org.apache.hudi.common.bootstrap.index.BootstrapIndex;
 import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
 import org.apache.hudi.common.table.timeline.HoodieInstant;
 import org.apache.hudi.common.table.timeline.HoodieTimeline;
@@ -38,7 +40,6 @@
 
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
-import org.apache.spark.api.java.JavaSparkContext;
 
 import java.io.IOException;
 import java.io.Serializable;
@@ -47,7 +48,7 @@
 import java.util.Objects;
 import java.util.stream.Collectors;
 
-public abstract class BaseRollbackActionExecutor extends BaseActionExecutor<HoodieRollbackMetadata> {
+public abstract class BaseRollbackActionExecutor<T extends HoodieRecordPayload, I, K, O> extends BaseActionExecutor<T, I, K, O, HoodieRollbackMetadata> {
 
   private static final Logger LOG = LogManager.getLogger(BaseRollbackActionExecutor.class);
 
@@ -61,25 +62,25 @@
   protected final boolean skipTimelinePublish;
   protected final boolean useMarkerBasedStrategy;
 
-  public BaseRollbackActionExecutor(JavaSparkContext jsc,
+  public BaseRollbackActionExecutor(HoodieEngineContext context,
       HoodieWriteConfig config,
-      HoodieTable<?> table,
+      HoodieTable<T, I, K, O> table,
       String instantTime,
       HoodieInstant instantToRollback,
       boolean deleteInstants) {
-    this(jsc, config, table, instantTime, instantToRollback, deleteInstants,
+    this(context, config, table, instantTime, instantToRollback, deleteInstants,
         false, config.shouldRollbackUsingMarkers());
   }
 
-  public BaseRollbackActionExecutor(JavaSparkContext jsc,
+  public BaseRollbackActionExecutor(HoodieEngineContext context,
       HoodieWriteConfig config,
-      HoodieTable<?> table,
+      HoodieTable<T, I, K, O> table,
       String instantTime,
       HoodieInstant instantToRollback,
       boolean deleteInstants,
       boolean skipTimelinePublish,
       boolean useMarkerBasedStrategy) {
-    super(jsc, config, table, instantTime);
+    super(context, config, table, instantTime);
     this.instantToRollback = instantToRollback;
     this.deleteInstants = deleteInstants;
     this.skipTimelinePublish = skipTimelinePublish;
@@ -90,13 +91,7 @@
     }
   }
 
-  protected RollbackStrategy getRollbackStrategy() {
-    if (useMarkerBasedStrategy) {
-      return new MarkerBasedRollbackStrategy(table, jsc, config, instantTime);
-    } else {
-      return this::executeRollbackUsingFileListing;
-    }
-  }
+  protected abstract RollbackStrategy getRollbackStrategy();
 
   protected abstract List<HoodieRollbackStat> executeRollback() throws IOException;
 
@@ -116,7 +111,7 @@
     }
 
     // Finally, remove the marker files post rollback.
-    new MarkerFiles(table, instantToRollback.getTimestamp()).quietDeleteMarkerDir(jsc, config.getMarkersDeleteParallelism());
+    new MarkerFiles(table, instantToRollback.getTimestamp()).quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism());
 
     return rollbackMetadata;
   }
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackRequest.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackRequest.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackRequest.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackRequest.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/rollback/RollbackUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/RollbackUtils.java
similarity index 96%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/rollback/RollbackUtils.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/RollbackUtils.java
index 3bfd645..897b448 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/rollback/RollbackUtils.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/RollbackUtils.java
@@ -20,9 +20,9 @@
 
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hudi.client.common.HoodieEngineContext;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
-import org.apache.spark.api.java.JavaSparkContext;
 
 import org.apache.hudi.common.HoodieRollbackStat;
 import org.apache.hudi.common.fs.FSUtils;
@@ -106,17 +106,17 @@
    *
    * @param instantToRollback Instant to Rollback
    * @param table instance of {@link HoodieTable} to use.
-   * @param jsc instance of {@link JavaSparkContext} to use.
+   * @param context instance of {@link HoodieEngineContext} to use.
    * @return list of rollback requests
    */
-  public static List<ListingBasedRollbackRequest> generateRollbackRequestsUsingFileListingMOR(HoodieInstant instantToRollback, HoodieTable table, JavaSparkContext jsc) throws IOException {
+  public static List<ListingBasedRollbackRequest> generateRollbackRequestsUsingFileListingMOR(HoodieInstant instantToRollback, HoodieTable table, HoodieEngineContext context) throws IOException {
     String commit = instantToRollback.getTimestamp();
     HoodieWriteConfig config = table.getConfig();
     List<String> partitions = FSUtils.getAllPartitionPaths(table.getMetaClient().getFs(), table.getMetaClient().getBasePath(),
         config.shouldAssumeDatePartitioning());
     int sparkPartitions = Math.max(Math.min(partitions.size(), config.getRollbackParallelism()), 1);
-    jsc.setJobGroup(RollbackUtils.class.getSimpleName(), "Generate all rollback requests");
-    return jsc.parallelize(partitions, Math.min(partitions.size(), sparkPartitions)).flatMap(partitionPath -> {
+    context.setJobStatus(RollbackUtils.class.getSimpleName(), "Generate all rollback requests");
+    return context.flatMap(partitions, partitionPath -> {
       HoodieActiveTimeline activeTimeline = table.getMetaClient().reloadActiveTimeline();
       List<ListingBasedRollbackRequest> partitionRollbackRequests = new ArrayList<>();
       switch (instantToRollback.getAction()) {
@@ -198,8 +198,8 @@
         default:
           break;
       }
-      return partitionRollbackRequests.iterator();
-    }).filter(Objects::nonNull).collect();
+      return partitionRollbackRequests.stream();
+    }, Math.min(partitions.size(), sparkPartitions)).stream().filter(Objects::nonNull).collect(Collectors.toList());
   }
 
   private static List<ListingBasedRollbackRequest> generateAppendRollbackBlocksAction(String partitionPath, HoodieInstant rollbackInstant,
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/savepoint/SavepointActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointActionExecutor.java
similarity index 86%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/savepoint/SavepointActionExecutor.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointActionExecutor.java
index ac95118..16fd9a4 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/savepoint/SavepointActionExecutor.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointActionExecutor.java
@@ -20,8 +20,10 @@
 
 import org.apache.hudi.avro.model.HoodieCleanMetadata;
 import org.apache.hudi.avro.model.HoodieSavepointMetadata;
+import org.apache.hudi.client.common.HoodieEngineContext;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.HoodieBaseFile;
+import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.model.HoodieTableType;
 import org.apache.hudi.common.table.timeline.HoodieInstant;
 import org.apache.hudi.common.table.timeline.HoodieTimeline;
@@ -35,7 +37,6 @@
 import org.apache.hudi.table.action.BaseActionExecutor;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
-import org.apache.spark.api.java.JavaSparkContext;
 import scala.Tuple2;
 
 import java.io.IOException;
@@ -43,20 +44,20 @@
 import java.util.Map;
 import java.util.stream.Collectors;
 
-public class SavepointActionExecutor extends BaseActionExecutor<HoodieSavepointMetadata> {
+public class SavepointActionExecutor<T extends HoodieRecordPayload, I, K, O> extends BaseActionExecutor<T, I, K, O, HoodieSavepointMetadata> {
 
   private static final Logger LOG = LogManager.getLogger(SavepointActionExecutor.class);
 
   private final String user;
   private final String comment;
 
-  public SavepointActionExecutor(JavaSparkContext jsc,
+  public SavepointActionExecutor(HoodieEngineContext context,
                                  HoodieWriteConfig config,
-                                 HoodieTable<?> table,
+                                 HoodieTable<T, I, K, O> table,
                                  String instantTime,
                                  String user,
                                  String comment) {
-    super(jsc, config, table, instantTime);
+    super(context, config, table, instantTime);
     this.user = user;
     this.comment = comment;
   }
@@ -87,19 +88,16 @@
       ValidationUtils.checkArgument(HoodieTimeline.compareTimestamps(instantTime, HoodieTimeline.GREATER_THAN_OR_EQUALS, lastCommitRetained),
           "Could not savepoint commit " + instantTime + " as this is beyond the lookup window " + lastCommitRetained);
 
-      jsc.setJobGroup(this.getClass().getSimpleName(), "Collecting latest files for savepoint " + instantTime);
-      Map<String, List<String>> latestFilesMap = jsc.parallelize(FSUtils.getAllPartitionPaths(table.getMetaClient().getFs(),
-          table.getMetaClient().getBasePath(), config.shouldAssumeDatePartitioning()))
-          .mapToPair(partitionPath -> {
+      context.setJobStatus(this.getClass().getSimpleName(), "Collecting latest files for savepoint " + instantTime);
+      Map<String, List<String>> latestFilesMap = context.mapToPair(FSUtils.getAllPartitionPaths(table.getMetaClient().getFs(),
+          table.getMetaClient().getBasePath(), config.shouldAssumeDatePartitioning()), partitionPath -> {
             // Scan all partitions files with this commit time
             LOG.info("Collecting latest files in partition path " + partitionPath);
             TableFileSystemView.BaseFileOnlyView view = table.getBaseFileOnlyView();
             List<String> latestFiles = view.getLatestBaseFilesBeforeOrOn(partitionPath, instantTime)
                 .map(HoodieBaseFile::getFileName).collect(Collectors.toList());
-            return new Tuple2<>(partitionPath, latestFiles);
-          })
-          .collectAsMap();
-
+          return new Tuple2<>(partitionPath, latestFiles);
+        }, null);
       HoodieSavepointMetadata metadata = TimelineMetadataUtils.convertSavepointMetadata(user, comment, latestFilesMap);
       // Nothing to save in the savepoint
       table.getActiveTimeline().createNewInstant(
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/savepoint/SavepointHelpers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointHelpers.java
similarity index 91%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/savepoint/SavepointHelpers.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointHelpers.java
index 06acd46..0d51a63 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/savepoint/SavepointHelpers.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointHelpers.java
@@ -32,7 +32,7 @@
 
   private static final Logger LOG = LogManager.getLogger(SavepointHelpers.class);
 
-  public static void deleteSavepoint(HoodieTable<?> table, String savepointTime) {
+  public static void deleteSavepoint(HoodieTable table, String savepointTime) {
     if (table.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ) {
       throw new UnsupportedOperationException("Savepointing is not supported or MergeOnRead table types");
     }
@@ -48,7 +48,7 @@
     LOG.info("Savepoint " + savepointTime + " deleted");
   }
 
-  public static void validateSavepointRestore(HoodieTable<?> table, String savepointTime) {
+  public static void validateSavepointRestore(HoodieTable table, String savepointTime) {
     // Make sure the restore was successful
     table.getMetaClient().reloadActiveTimeline();
     Option<HoodieInstant> lastInstant = table.getActiveTimeline()
@@ -61,7 +61,7 @@
             + lastInstant.get().getTimestamp());
   }
 
-  public static void validateSavepointPresence(HoodieTable<?> table, String savepointTime) {
+  public static void validateSavepointPresence(HoodieTable table, String savepointTime) {
     HoodieInstant savePoint = new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, savepointTime);
     boolean isSavepointPresent = table.getCompletedSavepointTimeline().containsInstant(savePoint);
     if (!isSavepointPresent) {
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/upgrade/UpgradeDowngrade.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/AbstractUpgradeDowngrade.java
similarity index 77%
rename from hudi-client/src/main/java/org/apache/hudi/table/upgrade/UpgradeDowngrade.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/AbstractUpgradeDowngrade.java
index 53af177..cafb816 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/upgrade/UpgradeDowngrade.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/AbstractUpgradeDowngrade.java
@@ -18,19 +18,18 @@
 
 package org.apache.hudi.table.upgrade;
 
+import org.apache.hudi.client.common.HoodieEngineContext;
 import org.apache.hudi.common.table.HoodieTableConfig;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
 import org.apache.hudi.common.table.HoodieTableVersion;
 import org.apache.hudi.common.util.FileIOUtils;
 import org.apache.hudi.config.HoodieWriteConfig;
-import org.apache.hudi.exception.HoodieUpgradeDowngradeException;
 
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
-import org.apache.spark.api.java.JavaSparkContext;
 
 import java.io.IOException;
 import java.util.Date;
@@ -39,14 +38,14 @@
 /**
  * Helper class to assist in upgrading/downgrading Hoodie when there is a version change.
  */
-public class UpgradeDowngrade {
+public abstract class AbstractUpgradeDowngrade {
 
-  private static final Logger LOG = LogManager.getLogger(UpgradeDowngrade.class);
+  private static final Logger LOG = LogManager.getLogger(AbstractUpgradeDowngrade.class);
   public static final String HOODIE_UPDATED_PROPERTY_FILE = "hoodie.properties.updated";
 
   private HoodieTableMetaClient metaClient;
-  private HoodieWriteConfig config;
-  private JavaSparkContext jsc;
+  protected HoodieWriteConfig config;
+  protected HoodieEngineContext context;
   private transient FileSystem fs;
   private Path updatedPropsFilePath;
   private Path propsFilePath;
@@ -73,28 +72,22 @@
    * @param metaClient instance of {@link HoodieTableMetaClient} to use
    * @param toVersion version to which upgrade or downgrade has to be done.
    * @param config instance of {@link HoodieWriteConfig} to use.
-   * @param jsc instance of {@link JavaSparkContext} to use.
+   * @param context instance of {@link HoodieEngineContext} to use.
    * @param instantTime current instant time that should not be touched.
    */
-  public static void run(HoodieTableMetaClient metaClient, HoodieTableVersion toVersion, HoodieWriteConfig config,
-                         JavaSparkContext jsc, String instantTime) {
-    try {
-      new UpgradeDowngrade(metaClient, config, jsc).run(toVersion, instantTime);
-    } catch (IOException e) {
-      throw new HoodieUpgradeDowngradeException("Error during upgrade/downgrade to version:" + toVersion, e);
-    }
-  }
+  public abstract void run(HoodieTableMetaClient metaClient, HoodieTableVersion toVersion, HoodieWriteConfig config,
+                         HoodieEngineContext context, String instantTime);
 
-  private UpgradeDowngrade(HoodieTableMetaClient metaClient, HoodieWriteConfig config, JavaSparkContext jsc) {
+  protected AbstractUpgradeDowngrade(HoodieTableMetaClient metaClient, HoodieWriteConfig config, HoodieEngineContext context) {
     this.metaClient = metaClient;
     this.config = config;
-    this.jsc = jsc;
+    this.context = context;
     this.fs = metaClient.getFs();
     this.updatedPropsFilePath = new Path(metaClient.getMetaPath(), HOODIE_UPDATED_PROPERTY_FILE);
     this.propsFilePath = new Path(metaClient.getMetaPath(), HoodieTableConfig.HOODIE_PROPERTIES_FILE);
   }
 
-  private void run(HoodieTableVersion toVersion, String instantTime) throws IOException {
+  protected void run(HoodieTableVersion toVersion, String instantTime) throws IOException {
     // Fetch version from property file and current version
     HoodieTableVersion fromVersion = metaClient.getTableConfig().getTableVersion();
     if (toVersion.versionCode() == fromVersion.versionCode()) {
@@ -144,19 +137,7 @@
     }
   }
 
-  private void upgrade(HoodieTableVersion fromVersion, HoodieTableVersion toVersion, String instantTime) {
-    if (fromVersion == HoodieTableVersion.ZERO && toVersion == HoodieTableVersion.ONE) {
-      new ZeroToOneUpgradeHandler().upgrade(config, jsc, instantTime);
-    } else {
-      throw new HoodieUpgradeDowngradeException(fromVersion.versionCode(), toVersion.versionCode(), true);
-    }
-  }
+  protected abstract void upgrade(HoodieTableVersion fromVersion, HoodieTableVersion toVersion, String instantTime);
 
-  private void downgrade(HoodieTableVersion fromVersion, HoodieTableVersion toVersion, String instantTime) {
-    if (fromVersion == HoodieTableVersion.ONE && toVersion == HoodieTableVersion.ZERO) {
-      new OneToZeroDowngradeHandler().downgrade(config, jsc, instantTime);
-    } else {
-      throw new HoodieUpgradeDowngradeException(fromVersion.versionCode(), toVersion.versionCode(), false);
-    }
-  }
+  protected abstract void downgrade(HoodieTableVersion fromVersion, HoodieTableVersion toVersion, String instantTime);
 }
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/upgrade/DowngradeHandler.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/DowngradeHandler.java
similarity index 84%
rename from hudi-client/src/main/java/org/apache/hudi/table/upgrade/DowngradeHandler.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/DowngradeHandler.java
index 948e44c..27389d9 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/upgrade/DowngradeHandler.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/DowngradeHandler.java
@@ -18,10 +18,9 @@
 
 package org.apache.hudi.table.upgrade;
 
+import org.apache.hudi.client.common.HoodieEngineContext;
 import org.apache.hudi.config.HoodieWriteConfig;
 
-import org.apache.spark.api.java.JavaSparkContext;
-
 /**
  * Interface to assist in downgrading Hoodie table.
  */
@@ -31,8 +30,8 @@
    * to be invoked to downgrade hoodie table from one version to a lower version.
    *
    * @param config instance of {@link HoodieWriteConfig} to be used.
-   * @param jsc instance of {@link JavaSparkContext} to be used.
+   * @param context instance of {@link HoodieEngineContext} to be used.
    * @param instantTime current instant time that should not touched.
    */
-  void downgrade(HoodieWriteConfig config, JavaSparkContext jsc, String instantTime);
+  void downgrade(HoodieWriteConfig config, HoodieEngineContext context, String instantTime);
 }
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/upgrade/UpgradeHandler.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/UpgradeHandler.java
similarity index 84%
rename from hudi-client/src/main/java/org/apache/hudi/table/upgrade/UpgradeHandler.java
rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/UpgradeHandler.java
index 4d56143..fdf1261 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/upgrade/UpgradeHandler.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/UpgradeHandler.java
@@ -18,10 +18,9 @@
 
 package org.apache.hudi.table.upgrade;
 
+import org.apache.hudi.client.common.HoodieEngineContext;
 import org.apache.hudi.config.HoodieWriteConfig;
 
-import org.apache.spark.api.java.JavaSparkContext;
-
 /**
  * Interface to assist in upgrading Hoodie table.
  */
@@ -31,8 +30,8 @@
    * to be invoked to upgrade hoodie table from one version to a higher version.
    *
    * @param config instance of {@link HoodieWriteConfig} to be used.
-   * @param jsc instance of {@link JavaSparkContext} to be used.
+   * @param context instance of {@link HoodieEngineContext} to be used.
    * @param instantTime current instant time that should not be touched.
    */
-  void upgrade(HoodieWriteConfig config, JavaSparkContext jsc, String instantTime);
+  void upgrade(HoodieWriteConfig config, HoodieEngineContext context, String instantTime);
 }
diff --git a/hudi-client/src/main/resources/log4j.properties b/hudi-client/hudi-client-common/src/main/resources/log4j.properties
similarity index 100%
copy from hudi-client/src/main/resources/log4j.properties
copy to hudi-client/hudi-client-common/src/main/resources/log4j.properties
diff --git a/hudi-client/src/test/java/org/apache/hudi/ClientFunctionalTestSuite.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/ClientFunctionalTestSuite.java
similarity index 100%
rename from hudi-client/src/test/java/org/apache/hudi/ClientFunctionalTestSuite.java
rename to hudi-client/hudi-client-common/src/test/java/org/apache/hudi/ClientFunctionalTestSuite.java
diff --git a/hudi-client/src/test/java/org/apache/hudi/callback/http/TestCallbackHttpClient.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/callback/http/TestCallbackHttpClient.java
similarity index 100%
rename from hudi-client/src/test/java/org/apache/hudi/callback/http/TestCallbackHttpClient.java
rename to hudi-client/hudi-client-common/src/test/java/org/apache/hudi/callback/http/TestCallbackHttpClient.java
diff --git a/hudi-client/src/test/java/org/apache/hudi/config/TestHoodieWriteConfig.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/config/TestHoodieWriteConfig.java
similarity index 100%
rename from hudi-client/src/test/java/org/apache/hudi/config/TestHoodieWriteConfig.java
rename to hudi-client/hudi-client-common/src/test/java/org/apache/hudi/config/TestHoodieWriteConfig.java
diff --git a/hudi-client/src/test/java/org/apache/hudi/metrics/TestHoodieConsoleMetrics.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/TestHoodieConsoleMetrics.java
similarity index 100%
rename from hudi-client/src/test/java/org/apache/hudi/metrics/TestHoodieConsoleMetrics.java
rename to hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/TestHoodieConsoleMetrics.java
diff --git a/hudi-client/src/test/java/org/apache/hudi/metrics/TestHoodieJmxMetrics.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/TestHoodieJmxMetrics.java
similarity index 100%
rename from hudi-client/src/test/java/org/apache/hudi/metrics/TestHoodieJmxMetrics.java
rename to hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/TestHoodieJmxMetrics.java
diff --git a/hudi-client/src/test/java/org/apache/hudi/metrics/TestHoodieMetrics.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/TestHoodieMetrics.java
similarity index 100%
rename from hudi-client/src/test/java/org/apache/hudi/metrics/TestHoodieMetrics.java
rename to hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/TestHoodieMetrics.java
diff --git a/hudi-client/src/test/java/org/apache/hudi/metrics/TestMetricsReporterFactory.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/TestMetricsReporterFactory.java
similarity index 100%
rename from hudi-client/src/test/java/org/apache/hudi/metrics/TestMetricsReporterFactory.java
rename to hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/TestMetricsReporterFactory.java
diff --git a/hudi-client/src/test/java/org/apache/hudi/metrics/datadog/TestDatadogHttpClient.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/datadog/TestDatadogHttpClient.java
similarity index 100%
rename from hudi-client/src/test/java/org/apache/hudi/metrics/datadog/TestDatadogHttpClient.java
rename to hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/datadog/TestDatadogHttpClient.java
diff --git a/hudi-client/src/test/java/org/apache/hudi/metrics/datadog/TestDatadogMetricsReporter.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/datadog/TestDatadogMetricsReporter.java
similarity index 100%
rename from hudi-client/src/test/java/org/apache/hudi/metrics/datadog/TestDatadogMetricsReporter.java
rename to hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/datadog/TestDatadogMetricsReporter.java
diff --git a/hudi-client/src/test/java/org/apache/hudi/metrics/datadog/TestDatadogReporter.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/datadog/TestDatadogReporter.java
similarity index 100%
rename from hudi-client/src/test/java/org/apache/hudi/metrics/datadog/TestDatadogReporter.java
rename to hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/datadog/TestDatadogReporter.java
diff --git a/hudi-client/src/test/java/org/apache/hudi/metrics/prometheus/TestPrometheusReporter.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/prometheus/TestPrometheusReporter.java
similarity index 100%
rename from hudi-client/src/test/java/org/apache/hudi/metrics/prometheus/TestPrometheusReporter.java
rename to hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/prometheus/TestPrometheusReporter.java
diff --git a/hudi-client/src/test/java/org/apache/hudi/metrics/prometheus/TestPushGateWayReporter.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/prometheus/TestPushGateWayReporter.java
similarity index 100%
rename from hudi-client/src/test/java/org/apache/hudi/metrics/prometheus/TestPushGateWayReporter.java
rename to hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/prometheus/TestPushGateWayReporter.java
diff --git a/hudi-client/src/test/java/org/apache/hudi/testutils/providers/DFSProvider.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/providers/DFSProvider.java
similarity index 100%
rename from hudi-client/src/test/java/org/apache/hudi/testutils/providers/DFSProvider.java
rename to hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/providers/DFSProvider.java
diff --git a/hudi-client/src/test/java/org/apache/hudi/testutils/providers/HoodieWriteClientProvider.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/providers/HoodieEngineContextProvider.java
similarity index 76%
copy from hudi-client/src/test/java/org/apache/hudi/testutils/providers/HoodieWriteClientProvider.java
copy to hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/providers/HoodieEngineContextProvider.java
index 4840af0..e876c08 100644
--- a/hudi-client/src/test/java/org/apache/hudi/testutils/providers/HoodieWriteClientProvider.java
+++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/providers/HoodieEngineContextProvider.java
@@ -19,12 +19,8 @@
 
 package org.apache.hudi.testutils.providers;
 
-import org.apache.hudi.client.HoodieWriteClient;
-import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.client.common.HoodieEngineContext;
 
-import java.io.IOException;
-
-public interface HoodieWriteClientProvider {
-
-  HoodieWriteClient getHoodieWriteClient(HoodieWriteConfig cfg) throws IOException;
+public interface HoodieEngineContextProvider {
+  HoodieEngineContext context();
 }
diff --git a/hudi-client/src/test/java/org/apache/hudi/testutils/providers/HoodieMetaClientProvider.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/providers/HoodieMetaClientProvider.java
similarity index 100%
rename from hudi-client/src/test/java/org/apache/hudi/testutils/providers/HoodieMetaClientProvider.java
rename to hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/providers/HoodieMetaClientProvider.java
diff --git a/hudi-client/src/test/java/org/apache/hudi/testutils/providers/HoodieWriteClientProvider.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/providers/HoodieWriteClientProvider.java
similarity index 86%
rename from hudi-client/src/test/java/org/apache/hudi/testutils/providers/HoodieWriteClientProvider.java
rename to hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/providers/HoodieWriteClientProvider.java
index 4840af0..9bc559d 100644
--- a/hudi-client/src/test/java/org/apache/hudi/testutils/providers/HoodieWriteClientProvider.java
+++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/providers/HoodieWriteClientProvider.java
@@ -19,12 +19,12 @@
 
 package org.apache.hudi.testutils.providers;
 
-import org.apache.hudi.client.HoodieWriteClient;
+import org.apache.hudi.client.AbstractHoodieWriteClient;
 import org.apache.hudi.config.HoodieWriteConfig;
 
 import java.io.IOException;
 
 public interface HoodieWriteClientProvider {
 
-  HoodieWriteClient getHoodieWriteClient(HoodieWriteConfig cfg) throws IOException;
+  AbstractHoodieWriteClient getHoodieWriteClient(HoodieWriteConfig cfg) throws IOException;
 }
diff --git a/hudi-client/src/test/resources/exampleEvolvedSchema.txt b/hudi-client/hudi-client-common/src/test/resources/exampleEvolvedSchema.txt
similarity index 100%
rename from hudi-client/src/test/resources/exampleEvolvedSchema.txt
rename to hudi-client/hudi-client-common/src/test/resources/exampleEvolvedSchema.txt
diff --git a/hudi-client/src/test/resources/exampleSchema.txt b/hudi-client/hudi-client-common/src/test/resources/exampleSchema.txt
similarity index 100%
rename from hudi-client/src/test/resources/exampleSchema.txt
rename to hudi-client/hudi-client-common/src/test/resources/exampleSchema.txt
diff --git a/hudi-client/src/test/resources/log4j-surefire-quiet.properties b/hudi-client/hudi-client-common/src/test/resources/log4j-surefire-quiet.properties
similarity index 100%
copy from hudi-client/src/test/resources/log4j-surefire-quiet.properties
copy to hudi-client/hudi-client-common/src/test/resources/log4j-surefire-quiet.properties
diff --git a/hudi-client/src/test/resources/log4j-surefire.properties b/hudi-client/hudi-client-common/src/test/resources/log4j-surefire.properties
similarity index 100%
copy from hudi-client/src/test/resources/log4j-surefire.properties
copy to hudi-client/hudi-client-common/src/test/resources/log4j-surefire.properties
diff --git a/hudi-client/hudi-spark-client/pom.xml b/hudi-client/hudi-spark-client/pom.xml
new file mode 100644
index 0000000..d99346d
--- /dev/null
+++ b/hudi-client/hudi-spark-client/pom.xml
@@ -0,0 +1,235 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <parent>
+    <artifactId>hudi-client</artifactId>
+    <groupId>org.apache.hudi</groupId>
+    <version>0.6.1-SNAPSHOT</version>
+  </parent>
+  <modelVersion>4.0.0</modelVersion>
+
+  <artifactId>hudi-spark-client</artifactId>
+  <version>${parent.version}</version>
+
+  <name>hudi-spark-client</name>
+  <packaging>jar</packaging>
+
+  <dependencies>
+    <!-- Hudi  -->
+    <dependency>
+      <groupId>org.apache.hudi</groupId>
+      <artifactId>hudi-client-common</artifactId>
+      <version>${parent.version}</version>
+    </dependency>
+
+    <!-- Spark -->
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-core_${scala.binary.version}</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-sql_${scala.binary.version}</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.spark</groupId>
+      <artifactId>spark-avro_${scala.binary.version}</artifactId>
+      <scope>provided</scope>
+    </dependency>
+
+    <!-- Parquet -->
+    <dependency>
+      <groupId>org.apache.parquet</groupId>
+      <artifactId>parquet-avro</artifactId>
+    </dependency>
+
+    <!-- Hoodie - Test -->
+    <dependency>
+      <groupId>org.apache.hudi</groupId>
+      <artifactId>hudi-common</artifactId>
+      <version>${project.version}</version>
+      <classifier>tests</classifier>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hudi</groupId>
+      <artifactId>hudi-client-common</artifactId>
+      <version>${project.version}</version>
+      <classifier>tests</classifier>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hudi</groupId>
+      <artifactId>hudi-hadoop-mr</artifactId>
+      <version>${project.version}</version>
+      <scope>test</scope>
+    </dependency>
+
+    <!-- HBase - Tests -->
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-testing-util</artifactId>
+      <version>${hbase.version}</version>
+      <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.codehaus.jackson</groupId>
+          <artifactId>jackson-mapper-asl</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>org.codehaus.jackson</groupId>
+          <artifactId>jackson-core-asl</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>javax.xml.bind</groupId>
+          <artifactId>*</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+
+    <!-- Hive - Tests -->
+    <dependency>
+      <groupId>${hive.groupid}</groupId>
+      <artifactId>hive-exec</artifactId>
+      <version>${hive.version}</version>
+      <scope>test</scope>
+      <classifier>${hive.exec.classifier}</classifier>
+    </dependency>
+    <dependency>
+      <groupId>${hive.groupid}</groupId>
+      <artifactId>hive-metastore</artifactId>
+      <version>${hive.version}</version>
+      <scope>test</scope>
+    </dependency>
+
+    <!-- Test -->
+    <dependency>
+      <groupId>org.junit.jupiter</groupId>
+      <artifactId>junit-jupiter-api</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.junit.jupiter</groupId>
+      <artifactId>junit-jupiter-engine</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.junit.vintage</groupId>
+      <artifactId>junit-vintage-engine</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.junit.jupiter</groupId>
+      <artifactId>junit-jupiter-params</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-junit-jupiter</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.junit.platform</groupId>
+      <artifactId>junit-platform-runner</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.junit.platform</groupId>
+      <artifactId>junit-platform-suite-api</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.junit.platform</groupId>
+      <artifactId>junit-platform-commons</artifactId>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.jacoco</groupId>
+        <artifactId>jacoco-maven-plugin</artifactId>
+      </plugin>
+      <plugin>
+        <groupId>net.alchim31.maven</groupId>
+        <artifactId>scala-maven-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>scala-compile-first</id>
+            <phase>process-resources</phase>
+            <goals>
+              <goal>add-source</goal>
+              <goal>compile</goal>
+            </goals>
+          </execution>
+          <execution>
+            <id>scala-test-compile</id>
+            <phase>process-test-resources</phase>
+            <goals>
+              <goal>testCompile</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <executions>
+          <execution>
+            <phase>compile</phase>
+            <goals>
+              <goal>compile</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+            <phase>test-compile</phase>
+          </execution>
+        </executions>
+        <configuration>
+          <skip>false</skip>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.rat</groupId>
+        <artifactId>apache-rat-plugin</artifactId>
+      </plugin>
+    </plugins>
+
+    <resources>
+      <resource>
+        <directory>src/main/resources</directory>
+      </resource>
+      <resource>
+        <directory>src/test/resources</directory>
+      </resource>
+    </resources>
+  </build>
+</project>
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/async/SparkAsyncCompactService.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/async/SparkAsyncCompactService.java
new file mode 100644
index 0000000..152a901
--- /dev/null
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/async/SparkAsyncCompactService.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.hudi.async;
+
+import org.apache.hudi.client.AbstractCompactor;
+import org.apache.hudi.client.AbstractHoodieWriteClient;
+import org.apache.hudi.client.HoodieSparkCompactor;
+import org.apache.hudi.client.common.HoodieEngineContext;
+
+public class SparkAsyncCompactService extends AsyncCompactService {
+
+  public SparkAsyncCompactService(HoodieEngineContext context, AbstractHoodieWriteClient client) {
+    super(context, client);
+  }
+
+  @Override
+  protected AbstractCompactor createCompactor(AbstractHoodieWriteClient client) {
+    return new HoodieSparkCompactor(client);
+  }
+}
diff --git a/hudi-client/src/main/java/org/apache/hudi/client/HoodieReadClient.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieReadClient.java
similarity index 82%
rename from hudi-client/src/main/java/org/apache/hudi/client/HoodieReadClient.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieReadClient.java
index 1a00197..4fb9f22 100644
--- a/hudi-client/src/main/java/org/apache/hudi/client/HoodieReadClient.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieReadClient.java
@@ -20,6 +20,7 @@
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hudi.avro.model.HoodieCompactionPlan;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.model.HoodieBaseFile;
 import org.apache.hudi.common.model.HoodieKey;
 import org.apache.hudi.common.model.HoodieRecord;
@@ -32,12 +33,13 @@
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.exception.HoodieIndexException;
 import org.apache.hudi.index.HoodieIndex;
+import org.apache.hudi.index.SparkHoodieIndex;
+import org.apache.hudi.table.HoodieSparkTable;
 import org.apache.hudi.table.HoodieTable;
 
 import org.apache.spark.SparkConf;
 import org.apache.spark.api.java.JavaPairRDD;
 import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
 import org.apache.spark.sql.Dataset;
 import org.apache.spark.sql.Row;
 import org.apache.spark.sql.SQLContext;
@@ -62,42 +64,42 @@
    * TODO: We need to persist the index type into hoodie.properties and be able to access the index just with a simple
    * basepath pointing to the table. Until, then just always assume a BloomIndex
    */
-  private final transient HoodieIndex<T> index;
-  private HoodieTable hoodieTable;
+  private final transient HoodieIndex<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> index;
+  private HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable;
   private transient Option<SQLContext> sqlContextOpt;
-  private final transient JavaSparkContext jsc;
+  private final transient HoodieSparkEngineContext context;
   private final transient Configuration hadoopConf;
 
   /**
    * @param basePath path to Hoodie table
    */
-  public HoodieReadClient(JavaSparkContext jsc, String basePath) {
-    this(jsc, HoodieWriteConfig.newBuilder().withPath(basePath)
+  public HoodieReadClient(HoodieSparkEngineContext context, String basePath) {
+    this(context, HoodieWriteConfig.newBuilder().withPath(basePath)
         // by default we use HoodieBloomIndex
         .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()).build());
   }
 
   /**
-   * @param jsc
+   * @param context
    * @param basePath
    * @param sqlContext
    */
-  public HoodieReadClient(JavaSparkContext jsc, String basePath, SQLContext sqlContext) {
-    this(jsc, basePath);
+  public HoodieReadClient(HoodieSparkEngineContext context, String basePath, SQLContext sqlContext) {
+    this(context, basePath);
     this.sqlContextOpt = Option.of(sqlContext);
   }
 
   /**
    * @param clientConfig instance of HoodieWriteConfig
    */
-  public HoodieReadClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig) {
-    this.jsc = jsc;
-    this.hadoopConf = jsc.hadoopConfiguration();
+  public HoodieReadClient(HoodieSparkEngineContext context, HoodieWriteConfig clientConfig) {
+    this.context = context;
+    this.hadoopConf = context.getHadoopConf().get();
     final String basePath = clientConfig.getBasePath();
     // Create a Hoodie table which encapsulated the commits and files visible
     HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf, basePath, true);
-    this.hoodieTable = HoodieTable.create(metaClient, clientConfig, hadoopConf);
-    this.index = HoodieIndex.createIndex(clientConfig);
+    this.hoodieTable = HoodieSparkTable.create(clientConfig, context, metaClient);
+    this.index = SparkHoodieIndex.createIndex(clientConfig);
     this.sqlContextOpt = Option.empty();
   }
 
@@ -134,8 +136,7 @@
    */
   public Dataset<Row> readROView(JavaRDD<HoodieKey> hoodieKeys, int parallelism) {
     assertSqlContext();
-    JavaPairRDD<HoodieKey, Option<Pair<String, String>>> lookupResultRDD =
-        index.fetchRecordLocation(hoodieKeys, jsc, hoodieTable);
+    JavaPairRDD<HoodieKey, Option<Pair<String, String>>> lookupResultRDD = checkExists(hoodieKeys);
     JavaPairRDD<HoodieKey, Option<String>> keyToFileRDD =
         lookupResultRDD.mapToPair(r -> new Tuple2<>(r._1, convertToDataFilePath(r._2)));
     List<String> paths = keyToFileRDD.filter(keyFileTuple -> keyFileTuple._2().isPresent())
@@ -161,8 +162,12 @@
    * FullFilePath value is not present, then the key is not found. If the FullFilePath value is present, it is the path
    * component (without scheme) of the URI underlying file
    */
-  public JavaPairRDD<HoodieKey, Option<String>> checkExists(JavaRDD<HoodieKey> hoodieKeys) {
-    return index.fetchRecordLocation(hoodieKeys, jsc, hoodieTable);
+  public JavaPairRDD<HoodieKey, Option<Pair<String, String>>> checkExists(JavaRDD<HoodieKey> hoodieKeys) {
+    return index.tagLocation(hoodieKeys.map(k -> new HoodieRecord<>(k, null)), context, hoodieTable)
+        .mapToPair(hr -> new Tuple2<>(hr.getKey(), hr.isCurrentLocationKnown()
+            ? Option.of(Pair.of(hr.getPartitionPath(), hr.getCurrentLocation().getFileId()))
+            : Option.empty())
+        );
   }
 
   /**
@@ -184,12 +189,12 @@
    * @return Tagged RDD of Hoodie records
    */
   public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> hoodieRecords) throws HoodieIndexException {
-    return index.tagLocation(hoodieRecords, jsc, hoodieTable);
+    return index.tagLocation(hoodieRecords, context, hoodieTable);
   }
 
   /**
    * Return all pending compactions with instant time for clients to decide what to compact next.
-   * 
+   *
    * @return
    */
   public List<Pair<String, HoodieCompactionPlan>> getPendingCompactions() {
diff --git a/hudi-client/src/main/java/org/apache/hudi/client/Compactor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieSparkCompactor.java
similarity index 75%
rename from hudi-client/src/main/java/org/apache/hudi/client/Compactor.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieSparkCompactor.java
index b235e25..68e6da7 100644
--- a/hudi-client/src/main/java/org/apache/hudi/client/Compactor.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieSparkCompactor.java
@@ -18,31 +18,27 @@
 
 package org.apache.hudi.client;
 
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.table.timeline.HoodieInstant;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.exception.HoodieException;
-
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
 import org.apache.spark.api.java.JavaRDD;
 
 import java.io.IOException;
-import java.io.Serializable;
 
-/**
- * Run one round of compaction.
- */
-public class Compactor implements Serializable {
+public class HoodieSparkCompactor<T extends HoodieRecordPayload> extends AbstractCompactor<T,
+    JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
+  private static final Logger LOG = LogManager.getLogger(HoodieSparkCompactor.class);
 
-  private static final long serialVersionUID = 1L;
-  private static final Logger LOG = LogManager.getLogger(Compactor.class);
-
-  private transient HoodieWriteClient compactionClient;
-
-  public Compactor(HoodieWriteClient compactionClient) {
-    this.compactionClient = compactionClient;
+  public HoodieSparkCompactor(AbstractHoodieWriteClient<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> compactionClient) {
+    super(compactionClient);
   }
 
+  @Override
   public void compact(HoodieInstant instant) throws IOException {
     LOG.info("Compactor executing compaction " + instant);
     JavaRDD<WriteStatus> res = compactionClient.compact(instant.getTimestamp());
diff --git a/hudi-client/src/main/java/org/apache/hudi/client/HoodieWriteResult.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieWriteResult.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/client/HoodieWriteResult.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieWriteResult.java
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java
new file mode 100644
index 0000000..56f0689
--- /dev/null
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java
@@ -0,0 +1,319 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.client;
+
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
+import org.apache.hudi.client.embedded.EmbeddedTimelineService;
+import org.apache.hudi.common.model.HoodieCommitMetadata;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.model.HoodieWriteStat;
+import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.HoodieTableVersion;
+import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieCommitException;
+import org.apache.hudi.index.HoodieIndex;
+import org.apache.hudi.index.SparkHoodieIndex;
+import org.apache.hudi.table.BulkInsertPartitioner;
+import org.apache.hudi.table.HoodieSparkTable;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.table.action.HoodieWriteMetadata;
+import org.apache.hudi.table.action.compact.SparkCompactHelpers;
+import org.apache.hudi.table.upgrade.SparkUpgradeDowngrade;
+
+import com.codahale.metrics.Timer;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.SparkConf;
+import org.apache.spark.api.java.JavaRDD;
+
+import java.io.IOException;
+import java.text.ParseException;
+import java.util.List;
+import java.util.Map;
+
+@SuppressWarnings("checkstyle:LineLength")
+public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
+    AbstractHoodieWriteClient<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
+
+  private static final Logger LOG = LogManager.getLogger(SparkRDDWriteClient.class);
+
+  public SparkRDDWriteClient(HoodieEngineContext context, HoodieWriteConfig clientConfig) {
+    super(context, clientConfig);
+  }
+
+  public SparkRDDWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig, boolean rollbackPending) {
+    super(context, writeConfig, rollbackPending);
+  }
+
+  public SparkRDDWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig, boolean rollbackPending,
+                             Option<EmbeddedTimelineService> timelineService) {
+    super(context, writeConfig, rollbackPending, timelineService);
+  }
+
+  /**
+   * Register hudi classes for Kryo serialization.
+   *
+   * @param conf instance of SparkConf
+   * @return SparkConf
+   */
+  public static SparkConf registerClasses(SparkConf conf) {
+    conf.registerKryoClasses(new Class[]{HoodieWriteConfig.class, HoodieRecord.class, HoodieKey.class});
+    return conf;
+  }
+
+  @Override
+  protected HoodieIndex<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> createIndex(HoodieWriteConfig writeConfig) {
+    return SparkHoodieIndex.createIndex(config);
+  }
+
+  /**
+   * Complete changes performed at the given instantTime marker with specified action.
+   */
+  @Override
+  public boolean commit(String instantTime, JavaRDD<WriteStatus> writeStatuses, Option<Map<String, String>> extraMetadata,
+                        String commitActionType, Map<String, List<String>> partitionToReplacedFileIds) {
+    List<HoodieWriteStat> writeStats = writeStatuses.map(WriteStatus::getStat).collect();
+    return commitStats(instantTime, writeStats, extraMetadata, commitActionType, partitionToReplacedFileIds);
+  }
+
+  @Override
+  protected HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> createTable(HoodieWriteConfig config,
+                                                                                                           Configuration hadoopConf) {
+    return HoodieSparkTable.create(config, context);
+  }
+
+  @Override
+  public JavaRDD<HoodieRecord<T>> filterExists(JavaRDD<HoodieRecord<T>> hoodieRecords) {
+    // Create a Hoodie table which encapsulated the commits and files visible
+    HoodieSparkTable<T> table = HoodieSparkTable.create(config, context);
+    Timer.Context indexTimer = metrics.getIndexCtx();
+    JavaRDD<HoodieRecord<T>> recordsWithLocation = getIndex().tagLocation(hoodieRecords, context, table);
+    metrics.updateIndexMetrics(LOOKUP_STR, metrics.getDurationInMs(indexTimer == null ? 0L : indexTimer.stop()));
+    return recordsWithLocation.filter(v1 -> !v1.isCurrentLocationKnown());
+  }
+
+  /**
+   * Main API to run bootstrap to hudi.
+   */
+  @Override
+  public void bootstrap(Option<Map<String, String>> extraMetadata) {
+    if (rollbackPending) {
+      rollBackInflightBootstrap();
+    }
+    getTableAndInitCtx(WriteOperationType.UPSERT, HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS).bootstrap(context, extraMetadata);
+  }
+
+  @Override
+  public JavaRDD<WriteStatus> upsert(JavaRDD<HoodieRecord<T>> records, String instantTime) {
+    HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table =
+        getTableAndInitCtx(WriteOperationType.UPSERT, instantTime);
+    table.validateUpsertSchema();
+    setOperationType(WriteOperationType.UPSERT);
+    this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime);
+    HoodieWriteMetadata<JavaRDD<WriteStatus>> result = table.upsert(context, instantTime, records);
+    if (result.getIndexLookupDuration().isPresent()) {
+      metrics.updateIndexMetrics(LOOKUP_STR, result.getIndexLookupDuration().get().toMillis());
+    }
+    return postWrite(result, instantTime, table);
+  }
+
+  @Override
+  public JavaRDD<WriteStatus> upsertPreppedRecords(JavaRDD<HoodieRecord<T>> preppedRecords, String instantTime) {
+    HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table =
+        getTableAndInitCtx(WriteOperationType.UPSERT_PREPPED, instantTime);
+    table.validateUpsertSchema();
+    setOperationType(WriteOperationType.UPSERT_PREPPED);
+    this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime);
+    HoodieWriteMetadata<JavaRDD<WriteStatus>> result = table.upsertPrepped(context,instantTime, preppedRecords);
+    return postWrite(result, instantTime, table);
+  }
+
+  @Override
+  public JavaRDD<WriteStatus> insert(JavaRDD<HoodieRecord<T>> records, String instantTime) {
+    HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table =
+        getTableAndInitCtx(WriteOperationType.INSERT, instantTime);
+    table.validateInsertSchema();
+    setOperationType(WriteOperationType.INSERT);
+    this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime);
+    HoodieWriteMetadata<JavaRDD<WriteStatus>> result = table.insert(context,instantTime, records);
+    return postWrite(result, instantTime, table);
+  }
+
+  @Override
+  public JavaRDD<WriteStatus> insertPreppedRecords(JavaRDD<HoodieRecord<T>> preppedRecords, String instantTime) {
+    HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table =
+        getTableAndInitCtx(WriteOperationType.INSERT_PREPPED, instantTime);
+    table.validateInsertSchema();
+    setOperationType(WriteOperationType.INSERT_PREPPED);
+    this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime);
+    HoodieWriteMetadata<JavaRDD<WriteStatus>> result = table.insertPrepped(context,instantTime, preppedRecords);
+    return postWrite(result, instantTime, table);
+  }
+
+  /**
+   * Removes all existing records from the partitions affected and inserts the given HoodieRecords, into the table.
+
+   * @param records HoodieRecords to insert
+   * @param instantTime Instant time of the commit
+   * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
+   */
+  public HoodieWriteResult insertOverwrite(JavaRDD<HoodieRecord<T>> records, final String instantTime) {
+    HoodieTable table = getTableAndInitCtx(WriteOperationType.INSERT_OVERWRITE, instantTime);
+    table.validateInsertSchema();
+    setOperationType(WriteOperationType.INSERT_OVERWRITE);
+    this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime);
+    HoodieWriteMetadata result = table.insertOverwrite(context, instantTime, records);
+    return new HoodieWriteResult(postWrite(result, instantTime, table), result.getPartitionToReplaceFileIds());
+  }
+
+  @Override
+  public JavaRDD<WriteStatus> bulkInsert(JavaRDD<HoodieRecord<T>> records, String instantTime) {
+    return bulkInsert(records, instantTime, Option.empty());
+  }
+
+  @Override
+  public JavaRDD<WriteStatus> bulkInsert(JavaRDD<HoodieRecord<T>> records, String instantTime, Option<BulkInsertPartitioner<JavaRDD<HoodieRecord<T>>>> userDefinedBulkInsertPartitioner) {
+    HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table =
+        getTableAndInitCtx(WriteOperationType.BULK_INSERT, instantTime);
+    table.validateInsertSchema();
+    setOperationType(WriteOperationType.BULK_INSERT);
+    this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime);
+    HoodieWriteMetadata<JavaRDD<WriteStatus>> result = table.bulkInsert(context,instantTime, records, userDefinedBulkInsertPartitioner);
+    return postWrite(result, instantTime, table);
+  }
+
+  @Override
+  public JavaRDD<WriteStatus> bulkInsertPreppedRecords(JavaRDD<HoodieRecord<T>> preppedRecords, String instantTime, Option<BulkInsertPartitioner<JavaRDD<HoodieRecord<T>>>> bulkInsertPartitioner) {
+    HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table =
+        getTableAndInitCtx(WriteOperationType.BULK_INSERT_PREPPED, instantTime);
+    table.validateInsertSchema();
+    setOperationType(WriteOperationType.BULK_INSERT_PREPPED);
+    this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime);
+    HoodieWriteMetadata<JavaRDD<WriteStatus>> result = table.bulkInsertPrepped(context,instantTime, preppedRecords, bulkInsertPartitioner);
+    return postWrite(result, instantTime, table);
+  }
+
+  @Override
+  public JavaRDD<WriteStatus> delete(JavaRDD<HoodieKey> keys, String instantTime) {
+    HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table = getTableAndInitCtx(WriteOperationType.DELETE, instantTime);
+    setOperationType(WriteOperationType.DELETE);
+    HoodieWriteMetadata<JavaRDD<WriteStatus>> result = table.delete(context,instantTime, keys);
+    return postWrite(result, instantTime, table);
+  }
+
+  @Override
+  protected JavaRDD<WriteStatus> postWrite(HoodieWriteMetadata<JavaRDD<WriteStatus>> result,
+                                           String instantTime,
+                                           HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) {
+    if (result.getIndexLookupDuration().isPresent()) {
+      metrics.updateIndexMetrics(getOperationType().name(), result.getIndexUpdateDuration().get().toMillis());
+    }
+    if (result.isCommitted()) {
+      // Perform post commit operations.
+      if (result.getFinalizeDuration().isPresent()) {
+        metrics.updateFinalizeWriteMetrics(result.getFinalizeDuration().get().toMillis(),
+            result.getWriteStats().get().size());
+      }
+
+      postCommit(hoodieTable, result.getCommitMetadata().get(), instantTime, Option.empty());
+
+      emitCommitMetrics(instantTime, result.getCommitMetadata().get(), hoodieTable.getMetaClient().getCommitActionType());
+    }
+    return result.getWriteStatuses();
+  }
+
+  @Override
+  public void commitCompaction(String compactionInstantTime, JavaRDD<WriteStatus> writeStatuses, Option<Map<String, String>> extraMetadata) throws IOException {
+    HoodieSparkTable<T> table = HoodieSparkTable.create(config, context);
+    HoodieCommitMetadata metadata = SparkCompactHelpers.newInstance().createCompactionMetadata(
+        table, compactionInstantTime, writeStatuses, config.getSchema());
+    extraMetadata.ifPresent(m -> m.forEach(metadata::addMetadata));
+    completeCompaction(metadata, writeStatuses, table, compactionInstantTime);
+  }
+
+  @Override
+  protected void completeCompaction(HoodieCommitMetadata metadata, JavaRDD<WriteStatus> writeStatuses,
+                                    HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table,
+                                    String compactionCommitTime) {
+    List<HoodieWriteStat> writeStats = writeStatuses.map(WriteStatus::getStat).collect();
+    finalizeWrite(table, compactionCommitTime, writeStats);
+    LOG.info("Committing Compaction " + compactionCommitTime + ". Finished with result " + metadata);
+    SparkCompactHelpers.newInstance().completeInflightCompaction(table, compactionCommitTime, metadata);
+
+    if (compactionTimer != null) {
+      long durationInMs = metrics.getDurationInMs(compactionTimer.stop());
+      try {
+        metrics.updateCommitMetrics(HoodieActiveTimeline.COMMIT_FORMATTER.parse(compactionCommitTime).getTime(),
+            durationInMs, metadata, HoodieActiveTimeline.COMPACTION_ACTION);
+      } catch (ParseException e) {
+        throw new HoodieCommitException("Commit time is not of valid format. Failed to commit compaction "
+            + config.getBasePath() + " at time " + compactionCommitTime, e);
+      }
+    }
+    LOG.info("Compacted successfully on commit " + compactionCommitTime);
+  }
+
+  @Override
+  protected JavaRDD<WriteStatus> compact(String compactionInstantTime, boolean shouldComplete) {
+    HoodieSparkTable<T> table = HoodieSparkTable.create(config, context);
+    HoodieTimeline pendingCompactionTimeline = table.getActiveTimeline().filterPendingCompactionTimeline();
+    HoodieInstant inflightInstant = HoodieTimeline.getCompactionInflightInstant(compactionInstantTime);
+    if (pendingCompactionTimeline.containsInstant(inflightInstant)) {
+      rollbackInflightCompaction(inflightInstant, table);
+      table.getMetaClient().reloadActiveTimeline();
+    }
+    compactionTimer = metrics.getCompactionCtx();
+    HoodieWriteMetadata<JavaRDD<WriteStatus>> compactionMetadata = table.compact(context, compactionInstantTime);
+    JavaRDD<WriteStatus> statuses = compactionMetadata.getWriteStatuses();
+    if (shouldComplete && compactionMetadata.getCommitMetadata().isPresent()) {
+      completeCompaction(compactionMetadata.getCommitMetadata().get(), statuses, table, compactionInstantTime);
+    }
+    return statuses;
+  }
+
+  @Override
+  protected HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> getTableAndInitCtx(WriteOperationType operationType, String instantTime) {
+    HoodieTableMetaClient metaClient = createMetaClient(true);
+    new SparkUpgradeDowngrade(metaClient, config, context).run(metaClient, HoodieTableVersion.current(), config, context, instantTime);
+    return getTableAndInitCtx(metaClient, operationType);
+  }
+
+  private HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> getTableAndInitCtx(HoodieTableMetaClient metaClient, WriteOperationType operationType) {
+    if (operationType == WriteOperationType.DELETE) {
+      setWriteSchemaForDeletes(metaClient);
+    }
+    // Create a Hoodie table which encapsulated the commits and files visible
+    HoodieSparkTable<T> table = HoodieSparkTable.create(config, (HoodieSparkEngineContext) context, metaClient);
+    if (table.getMetaClient().getCommitActionType().equals(HoodieTimeline.COMMIT_ACTION)) {
+      writeTimer = metrics.getCommitCtx();
+    } else {
+      writeTimer = metrics.getDeltaCommitCtx();
+    }
+    return table;
+  }
+}
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkTaskContextSupplier.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkTaskContextSupplier.java
new file mode 100644
index 0000000..563bc1a
--- /dev/null
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkTaskContextSupplier.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.hudi.client;
+
+import org.apache.hudi.client.common.EngineProperty;
+import org.apache.hudi.client.common.TaskContextSupplier;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+
+import org.apache.spark.SparkEnv;
+import org.apache.spark.TaskContext;
+import org.apache.spark.util.Utils;
+
+import java.io.Serializable;
+import java.util.function.Supplier;
+
+/**
+ * Spark task context supplier.
+ */
+public class SparkTaskContextSupplier extends TaskContextSupplier implements Serializable {
+
+  @Override
+  public Supplier<Integer> getPartitionIdSupplier() {
+    return TaskContext::getPartitionId;
+  }
+
+  @Override
+  public Supplier<Integer> getStageIdSupplier() {
+    return () -> TaskContext.get().stageId();
+  }
+
+  @Override
+  public Supplier<Long> getAttemptIdSupplier() {
+    return () -> TaskContext.get().taskAttemptId();
+  }
+
+  @Override
+  public Option<String> getProperty(EngineProperty prop) {
+    if (prop == EngineProperty.TOTAL_MEMORY_AVAILABLE) {
+      // This is hard-coded in spark code {@link
+      // https://github.com/apache/spark/blob/576c43fb4226e4efa12189b41c3bc862019862c6/core/src/main/scala/org/apache/
+      // spark/SparkContext.scala#L471} so have to re-define this here
+      final String DEFAULT_SPARK_EXECUTOR_MEMORY_MB = "1024"; // in MB
+      final String SPARK_EXECUTOR_MEMORY_PROP = "spark.executor.memory";
+      if (SparkEnv.get() != null) {
+        // 1 GB is the default conf used by Spark, look at SparkContext.scala
+        return Option.ofNullable(String.valueOf(Utils.memoryStringToMb(SparkEnv.get().conf()
+            .get(SPARK_EXECUTOR_MEMORY_PROP, DEFAULT_SPARK_EXECUTOR_MEMORY_MB)) * 1024 * 1024L));
+      }
+      return Option.empty();
+    } else if (prop == EngineProperty.MEMORY_FRACTION_IN_USE) {
+      // This is hard-coded in spark code {@link
+      // https://github.com/apache/spark/blob/576c43fb4226e4efa12189b41c3bc862019862c6/core/src/main/scala/org/apache/
+      // spark/memory/UnifiedMemoryManager.scala#L231} so have to re-define this here
+      final String DEFAULT_SPARK_EXECUTOR_MEMORY_FRACTION = "0.6";
+      final String SPARK_EXECUTOR_MEMORY_FRACTION_PROP = "spark.memory.fraction";
+      if (SparkEnv.get() != null) {
+        // 0.6 is the default value used by Spark,
+        // look at {@link
+        // https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/SparkConf.scala#L507}
+        return Option.ofNullable(SparkEnv.get().conf()
+            .get(SPARK_EXECUTOR_MEMORY_FRACTION_PROP, DEFAULT_SPARK_EXECUTOR_MEMORY_FRACTION));
+      }
+      return Option.empty();
+    }
+    throw new HoodieException("Unknown engine property :" + prop);
+  }
+}
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/bootstrap/HoodieSparkBootstrapSchemaProvider.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/bootstrap/HoodieSparkBootstrapSchemaProvider.java
new file mode 100644
index 0000000..30cde59
--- /dev/null
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/bootstrap/HoodieSparkBootstrapSchemaProvider.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.hudi.client.bootstrap;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.avro.model.HoodieFileStatus;
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.common.bootstrap.FileStatusUtils;
+import org.apache.hudi.common.util.ParquetUtils;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.parquet.schema.MessageType;
+import org.apache.spark.sql.avro.SchemaConverters;
+import org.apache.spark.sql.execution.datasources.parquet.ParquetToSparkSchemaConverter;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.types.StructType;
+
+import java.util.List;
+import java.util.Objects;
+
+public class HoodieSparkBootstrapSchemaProvider extends HoodieBootstrapSchemaProvider {
+  public HoodieSparkBootstrapSchemaProvider(HoodieWriteConfig writeConfig) {
+    super(writeConfig);
+  }
+
+  @Override
+  protected Schema getBootstrapSourceSchema(HoodieEngineContext context, List<Pair<String, List<HoodieFileStatus>>> partitions) {
+    MessageType parquetSchema = partitions.stream().flatMap(p -> p.getValue().stream()).map(fs -> {
+      try {
+        Path filePath = FileStatusUtils.toPath(fs.getPath());
+        return ParquetUtils.readSchema(context.getHadoopConf().get(), filePath);
+      } catch (Exception ex) {
+        return null;
+      }
+    }).filter(Objects::nonNull).findAny()
+        .orElseThrow(() -> new HoodieException("Could not determine schema from the data files."));
+
+
+    ParquetToSparkSchemaConverter converter = new ParquetToSparkSchemaConverter(
+        Boolean.parseBoolean(SQLConf.PARQUET_BINARY_AS_STRING().defaultValueString()),
+        Boolean.parseBoolean(SQLConf.PARQUET_INT96_AS_TIMESTAMP().defaultValueString()));
+    StructType sparkSchema = converter.convert(parquetSchema);
+    String tableName = HoodieAvroUtils.sanitizeName(writeConfig.getTableName());
+    String structName = tableName + "_record";
+    String recordNamespace = "hoodie." + tableName;
+
+    return SchemaConverters.toAvroType(sparkSchema, false, structName, recordNamespace);
+  }
+}
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/common/HoodieSparkEngineContext.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/common/HoodieSparkEngineContext.java
new file mode 100644
index 0000000..0f17511
--- /dev/null
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/common/HoodieSparkEngineContext.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.hudi.client.common;
+
+import org.apache.hudi.client.SparkTaskContextSupplier;
+import org.apache.hudi.common.config.SerializableConfiguration;
+import org.apache.hudi.client.common.function.SerializableConsumer;
+import org.apache.hudi.client.common.function.SerializableFunction;
+import org.apache.hudi.client.common.function.SerializablePairFunction;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.SQLContext;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.stream.Stream;
+
+/**
+ * A Spark engine implementation of HoodieEngineContext.
+ */
+public class HoodieSparkEngineContext extends HoodieEngineContext {
+
+  private final JavaSparkContext javaSparkContext;
+  private SQLContext sqlContext;
+
+  public HoodieSparkEngineContext(JavaSparkContext jsc) {
+    super(new SerializableConfiguration(jsc.hadoopConfiguration()), new SparkTaskContextSupplier());
+    this.javaSparkContext = jsc;
+    this.sqlContext = SQLContext.getOrCreate(jsc.sc());
+  }
+
+  public void setSqlContext(SQLContext sqlContext) {
+    this.sqlContext = sqlContext;
+  }
+
+  public JavaSparkContext getJavaSparkContext() {
+    return javaSparkContext;
+  }
+
+  public SQLContext getSqlContext() {
+    return sqlContext;
+  }
+
+  public static JavaSparkContext getSparkContext(HoodieEngineContext context) {
+    return ((HoodieSparkEngineContext) context).getJavaSparkContext();
+  }
+
+  @Override
+  public <I, O> List<O> map(List<I> data, SerializableFunction<I, O> func, int parallelism) {
+    return javaSparkContext.parallelize(data, parallelism).map(func::apply).collect();
+  }
+
+  @Override
+  public <I, O> List<O> flatMap(List<I> data, SerializableFunction<I, Stream<O>> func, int parallelism) {
+    return javaSparkContext.parallelize(data, parallelism).flatMap(x -> func.apply(x).iterator()).collect();
+  }
+
+  @Override
+  public <I> void foreach(List<I> data, SerializableConsumer<I> consumer, int parallelism) {
+    javaSparkContext.parallelize(data, parallelism).foreach(consumer::accept);
+  }
+
+  @Override
+  public <I, K, V> Map<K, V> mapToPair(List<I> data, SerializablePairFunction<I, K, V> func, Integer parallelism) {
+    if (Objects.nonNull(parallelism)) {
+      return javaSparkContext.parallelize(data, parallelism).mapToPair(func::call).collectAsMap();
+    } else {
+      return javaSparkContext.parallelize(data).mapToPair(func::call).collectAsMap();
+    }
+  }
+
+  @Override
+  public void setProperty(EngineProperty key, String value) {
+    if (key == EngineProperty.COMPACTION_POOL_NAME) {
+      javaSparkContext.setLocalProperty("spark.scheduler.pool", value);
+    } else {
+      throw new HoodieException("Unknown engine property :" + key);
+    }
+  }
+
+  @Override
+  public Option<String> getProperty(EngineProperty key) {
+    if (key == EngineProperty.EMBEDDED_SERVER_HOST) {
+      return Option.ofNullable(javaSparkContext.getConf().get("spark.driver.host", null));
+    }
+    throw new HoodieException("Unknown engine property :" + key);
+  }
+
+  @Override
+  public void setJobStatus(String activeModule, String activityDescription) {
+    javaSparkContext.setJobGroup(activeModule, activityDescription);
+  }
+}
diff --git a/hudi-client/src/main/java/org/apache/hudi/client/model/HoodieInternalRow.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/model/HoodieInternalRow.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/client/model/HoodieInternalRow.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/model/HoodieInternalRow.java
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkMemoryUtils.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkMemoryUtils.java
new file mode 100644
index 0000000..0fa75dc
--- /dev/null
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkMemoryUtils.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.hudi.client.utils;
+
+import org.apache.hudi.config.HoodieIndexConfig;
+
+import org.apache.spark.storage.StorageLevel;
+
+import java.util.Properties;
+
+import static org.apache.hudi.config.HoodieWriteConfig.WRITE_STATUS_STORAGE_LEVEL;
+
+/**
+ * Spark config utils.
+ */
+public class SparkMemoryUtils {
+  public static StorageLevel getWriteStatusStorageLevel(Properties properties) {
+    return StorageLevel.fromString(properties.getProperty(WRITE_STATUS_STORAGE_LEVEL));
+  }
+
+  public static StorageLevel getBloomIndexInputStorageLevel(Properties properties) {
+    return StorageLevel.fromString(properties.getProperty(HoodieIndexConfig.BLOOM_INDEX_INPUT_STORAGE_LEVEL));
+  }
+
+  public static StorageLevel getSimpleIndexInputStorageLevel(Properties properties) {
+    return StorageLevel.fromString(properties.getProperty(HoodieIndexConfig.SIMPLE_INDEX_INPUT_STORAGE_LEVEL));
+  }
+}
diff --git a/hudi-client/src/main/java/org/apache/hudi/execution/SparkBoundedInMemoryExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkBoundedInMemoryExecutor.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/execution/SparkBoundedInMemoryExecutor.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkBoundedInMemoryExecutor.java
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.java
new file mode 100644
index 0000000..ec90ef8
--- /dev/null
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/SparkLazyInsertIterable.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.hudi.execution;
+
+import org.apache.avro.Schema;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.TaskContextSupplier;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.io.WriteHandleFactory;
+import org.apache.hudi.table.HoodieTable;
+
+import java.util.Iterator;
+import java.util.List;
+
+public class SparkLazyInsertIterable<T extends HoodieRecordPayload> extends HoodieLazyInsertIterable<T> {
+
+  public SparkLazyInsertIterable(Iterator<HoodieRecord<T>> recordItr,
+                                 boolean areRecordsSorted,
+                                 HoodieWriteConfig config,
+                                 String instantTime,
+                                 HoodieTable hoodieTable,
+                                 String idPrefix,
+                                 TaskContextSupplier taskContextSupplier) {
+    super(recordItr, areRecordsSorted, config, instantTime, hoodieTable, idPrefix, taskContextSupplier);
+  }
+
+  public SparkLazyInsertIterable(Iterator<HoodieRecord<T>> recordItr,
+                                 boolean areRecordsSorted,
+                                 HoodieWriteConfig config,
+                                 String instantTime,
+                                 HoodieTable hoodieTable,
+                                 String idPrefix,
+                                 TaskContextSupplier taskContextSupplier,
+                                 WriteHandleFactory writeHandleFactory) {
+    super(recordItr, areRecordsSorted, config, instantTime, hoodieTable, idPrefix, taskContextSupplier, writeHandleFactory);
+  }
+
+  @Override
+  protected List<WriteStatus> computeNext() {
+    // Executor service used for launching writer thread.
+    BoundedInMemoryExecutor<HoodieRecord<T>, HoodieInsertValueGenResult<HoodieRecord>, List<WriteStatus>> bufferedIteratorExecutor =
+        null;
+    try {
+      final Schema schema = new Schema.Parser().parse(hoodieConfig.getSchema());
+      bufferedIteratorExecutor =
+          new SparkBoundedInMemoryExecutor<>(hoodieConfig, inputItr, getInsertHandler(), getTransformFunction(schema));
+      final List<WriteStatus> result = bufferedIteratorExecutor.execute();
+      assert result != null && !result.isEmpty() && !bufferedIteratorExecutor.isRemaining();
+      return result;
+    } catch (Exception e) {
+      throw new HoodieException(e);
+    } finally {
+      if (null != bufferedIteratorExecutor) {
+        bufferedIteratorExecutor.shutdownNow();
+      }
+    }
+  }
+}
diff --git a/hudi-client/src/main/java/org/apache/hudi/execution/bulkinsert/BulkInsertInternalPartitionerFactory.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/BulkInsertInternalPartitionerFactory.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/execution/bulkinsert/BulkInsertInternalPartitionerFactory.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/BulkInsertInternalPartitionerFactory.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/execution/bulkinsert/BulkInsertMapFunction.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/BulkInsertMapFunction.java
similarity index 87%
rename from hudi-client/src/main/java/org/apache/hudi/execution/bulkinsert/BulkInsertMapFunction.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/BulkInsertMapFunction.java
index 71c10ed..db73a9c 100644
--- a/hudi-client/src/main/java/org/apache/hudi/execution/bulkinsert/BulkInsertMapFunction.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/BulkInsertMapFunction.java
@@ -22,7 +22,7 @@
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.config.HoodieWriteConfig;
-import org.apache.hudi.execution.LazyInsertIterable;
+import org.apache.hudi.execution.SparkLazyInsertIterable;
 import org.apache.hudi.table.HoodieTable;
 
 import org.apache.spark.api.java.function.Function2;
@@ -39,11 +39,11 @@
   private String instantTime;
   private boolean areRecordsSorted;
   private HoodieWriteConfig config;
-  private HoodieTable<T> hoodieTable;
+  private HoodieTable hoodieTable;
   private List<String> fileIDPrefixes;
 
   public BulkInsertMapFunction(String instantTime, boolean areRecordsSorted,
-                               HoodieWriteConfig config, HoodieTable<T> hoodieTable,
+                               HoodieWriteConfig config, HoodieTable hoodieTable,
                                List<String> fileIDPrefixes) {
     this.instantTime = instantTime;
     this.areRecordsSorted = areRecordsSorted;
@@ -54,7 +54,7 @@
 
   @Override
   public Iterator<List<WriteStatus>> call(Integer partition, Iterator<HoodieRecord<T>> recordItr) {
-    return new LazyInsertIterable<>(recordItr, areRecordsSorted, config, instantTime, hoodieTable,
-        fileIDPrefixes.get(partition), hoodieTable.getSparkTaskContextSupplier());
+    return new SparkLazyInsertIterable<>(recordItr, areRecordsSorted, config, instantTime, hoodieTable,
+        fileIDPrefixes.get(partition), hoodieTable.getTaskContextSupplier());
   }
 }
diff --git a/hudi-client/src/main/java/org/apache/hudi/execution/bulkinsert/GlobalSortPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/GlobalSortPartitioner.java
similarity index 96%
rename from hudi-client/src/main/java/org/apache/hudi/execution/bulkinsert/GlobalSortPartitioner.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/GlobalSortPartitioner.java
index 3f4077a..a184c00 100644
--- a/hudi-client/src/main/java/org/apache/hudi/execution/bulkinsert/GlobalSortPartitioner.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/GlobalSortPartitioner.java
@@ -32,7 +32,7 @@
  * @param <T> HoodieRecordPayload type
  */
 public class GlobalSortPartitioner<T extends HoodieRecordPayload>
-    implements BulkInsertPartitioner<T> {
+    implements BulkInsertPartitioner<JavaRDD<HoodieRecord<T>>> {
 
   @Override
   public JavaRDD<HoodieRecord<T>> repartitionRecords(JavaRDD<HoodieRecord<T>> records,
diff --git a/hudi-client/src/main/java/org/apache/hudi/execution/bulkinsert/NonSortPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/NonSortPartitioner.java
similarity index 95%
rename from hudi-client/src/main/java/org/apache/hudi/execution/bulkinsert/NonSortPartitioner.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/NonSortPartitioner.java
index 571b8aa..19c90ec 100644
--- a/hudi-client/src/main/java/org/apache/hudi/execution/bulkinsert/NonSortPartitioner.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/NonSortPartitioner.java
@@ -31,7 +31,7 @@
  * @param <T> HoodieRecordPayload type
  */
 public class NonSortPartitioner<T extends HoodieRecordPayload>
-    implements BulkInsertPartitioner<T> {
+    implements BulkInsertPartitioner<JavaRDD<HoodieRecord<T>>> {
 
   @Override
   public JavaRDD<HoodieRecord<T>> repartitionRecords(JavaRDD<HoodieRecord<T>> records,
diff --git a/hudi-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDPartitionSortPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDPartitionSortPartitioner.java
similarity index 97%
rename from hudi-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDPartitionSortPartitioner.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDPartitionSortPartitioner.java
index 9fc91a4..9526ad5 100644
--- a/hudi-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDPartitionSortPartitioner.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDPartitionSortPartitioner.java
@@ -38,7 +38,7 @@
  * @param <T> HoodieRecordPayload type
  */
 public class RDDPartitionSortPartitioner<T extends HoodieRecordPayload>
-    implements BulkInsertPartitioner<T> {
+    implements BulkInsertPartitioner<JavaRDD<HoodieRecord<T>>> {
 
   @Override
   public JavaRDD<HoodieRecord<T>> repartitionRecords(JavaRDD<HoodieRecord<T>> records,
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkHoodieIndex.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkHoodieIndex.java
new file mode 100644
index 0000000..dd73bf2
--- /dev/null
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkHoodieIndex.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.hudi.index;
+
+import org.apache.hudi.ApiMaturityLevel;
+import org.apache.hudi.PublicAPIMethod;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.util.ReflectionUtils;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieIndexException;
+import org.apache.hudi.index.bloom.SparkHoodieBloomIndex;
+import org.apache.hudi.index.bloom.SparkHoodieGlobalBloomIndex;
+import org.apache.hudi.index.hbase.SparkHoodieHBaseIndex;
+import org.apache.hudi.index.simple.SparkHoodieGlobalSimpleIndex;
+import org.apache.hudi.index.simple.SparkHoodieSimpleIndex;
+import org.apache.hudi.table.HoodieTable;
+
+import org.apache.spark.api.java.JavaRDD;
+
+@SuppressWarnings("checkstyle:LineLength")
+public abstract class SparkHoodieIndex<T extends HoodieRecordPayload> extends HoodieIndex<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
+  protected SparkHoodieIndex(HoodieWriteConfig config) {
+    super(config);
+  }
+
+  public static SparkHoodieIndex createIndex(HoodieWriteConfig config) {
+    // first use index class config to create index.
+    if (!StringUtils.isNullOrEmpty(config.getIndexClass())) {
+      Object instance = ReflectionUtils.loadClass(config.getIndexClass(), config);
+      if (!(instance instanceof HoodieIndex)) {
+        throw new HoodieIndexException(config.getIndexClass() + " is not a subclass of HoodieIndex");
+      }
+      return (SparkHoodieIndex) instance;
+    }
+    switch (config.getIndexType()) {
+      case HBASE:
+        return new SparkHoodieHBaseIndex<>(config);
+      case INMEMORY:
+        return new SparkInMemoryHashIndex(config);
+      case BLOOM:
+        return new SparkHoodieBloomIndex<>(config);
+      case GLOBAL_BLOOM:
+        return new SparkHoodieGlobalBloomIndex<>(config);
+      case SIMPLE:
+        return new SparkHoodieSimpleIndex(config);
+      case GLOBAL_SIMPLE:
+        return new SparkHoodieGlobalSimpleIndex(config);
+      default:
+        throw new HoodieIndexException("Index type unspecified, set " + config.getIndexType());
+    }
+  }
+
+  @Override
+  @PublicAPIMethod(maturity = ApiMaturityLevel.STABLE)
+  public abstract JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
+                                                      HoodieEngineContext context,
+                                                      HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) throws HoodieIndexException;
+
+  @Override
+  @PublicAPIMethod(maturity = ApiMaturityLevel.STABLE)
+  public abstract JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> records,
+                                                       HoodieEngineContext context,
+                                                       HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) throws HoodieIndexException;
+}
diff --git a/hudi-client/src/main/java/org/apache/hudi/index/InMemoryHashIndex.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkInMemoryHashIndex.java
similarity index 69%
rename from hudi-client/src/main/java/org/apache/hudi/index/InMemoryHashIndex.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkInMemoryHashIndex.java
index 49512ab..55ce8d2 100644
--- a/hudi-client/src/main/java/org/apache/hudi/index/InMemoryHashIndex.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkInMemoryHashIndex.java
@@ -19,19 +19,16 @@
 package org.apache.hudi.index;
 
 import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieEngineContext;
 import org.apache.hudi.common.model.HoodieKey;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieRecordLocation;
 import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.util.Option;
-import org.apache.hudi.common.util.collection.Pair;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.table.HoodieTable;
 
-import org.apache.spark.api.java.JavaPairRDD;
 import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
-import org.apache.spark.api.java.function.Function;
 import org.apache.spark.api.java.function.Function2;
 
 import java.util.ArrayList;
@@ -45,13 +42,14 @@
  * <p>
  * ONLY USE FOR LOCAL TESTING
  */
-public class InMemoryHashIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
+@SuppressWarnings("checkstyle:LineLength")
+public class SparkInMemoryHashIndex<T extends HoodieRecordPayload> extends SparkHoodieIndex<T> {
 
   private static ConcurrentMap<HoodieKey, HoodieRecordLocation> recordLocationMap;
 
-  public InMemoryHashIndex(HoodieWriteConfig config) {
+  public SparkInMemoryHashIndex(HoodieWriteConfig config) {
     super(config);
-    synchronized (InMemoryHashIndex.class) {
+    synchronized (SparkInMemoryHashIndex.class) {
       if (recordLocationMap == null) {
         recordLocationMap = new ConcurrentHashMap<>();
       }
@@ -59,37 +57,29 @@
   }
 
   @Override
-  public JavaPairRDD<HoodieKey, Option<Pair<String, String>>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys,
-      JavaSparkContext jsc, HoodieTable<T> hoodieTable) {
-    throw new UnsupportedOperationException("InMemory index does not implement check exist yet");
-  }
-
-  @Override
-  public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, JavaSparkContext jsc,
-      HoodieTable<T> hoodieTable) {
+  public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, HoodieEngineContext context,
+                                              HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) {
     return recordRDD.mapPartitionsWithIndex(this.new LocationTagFunction(), true);
   }
 
   @Override
-  public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, JavaSparkContext jsc,
-      HoodieTable<T> hoodieTable) {
-    return writeStatusRDD.map(new Function<WriteStatus, WriteStatus>() {
-      @Override
-      public WriteStatus call(WriteStatus writeStatus) {
-        for (HoodieRecord record : writeStatus.getWrittenRecords()) {
-          if (!writeStatus.isErrored(record.getKey())) {
-            HoodieKey key = record.getKey();
-            Option<HoodieRecordLocation> newLocation = record.getNewLocation();
-            if (newLocation.isPresent()) {
-              recordLocationMap.put(key, newLocation.get());
-            } else {
-              // Delete existing index for a deleted record
-              recordLocationMap.remove(key);
-            }
+  public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
+                                             HoodieEngineContext context,
+                                             HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) {
+    return writeStatusRDD.map(writeStatus -> {
+      for (HoodieRecord record : writeStatus.getWrittenRecords()) {
+        if (!writeStatus.isErrored(record.getKey())) {
+          HoodieKey key = record.getKey();
+          Option<HoodieRecordLocation> newLocation = record.getNewLocation();
+          if (newLocation.isPresent()) {
+            recordLocationMap.put(key, newLocation.get());
+          } else {
+            // Delete existing index for a deleted record
+            recordLocationMap.remove(key);
           }
         }
-        return writeStatus;
       }
+      return writeStatus;
     });
   }
 
diff --git a/hudi-client/src/main/java/org/apache/hudi/index/bloom/BucketizedBloomCheckPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/BucketizedBloomCheckPartitioner.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/index/bloom/BucketizedBloomCheckPartitioner.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/BucketizedBloomCheckPartitioner.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndexCheckFunction.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndexCheckFunction.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndexCheckFunction.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndexCheckFunction.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndex.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieBloomIndex.java
similarity index 81%
rename from hudi-client/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndex.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieBloomIndex.java
index e03f38a..894b41b 100644
--- a/hudi-client/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndex.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieBloomIndex.java
@@ -19,7 +19,8 @@
 package org.apache.hudi.index.bloom;
 
 import org.apache.hudi.client.WriteStatus;
-import org.apache.hudi.client.utils.SparkConfigUtils;
+import org.apache.hudi.client.utils.SparkMemoryUtils;
+import org.apache.hudi.client.common.HoodieEngineContext;
 import org.apache.hudi.common.model.HoodieKey;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieRecordLocation;
@@ -28,8 +29,8 @@
 import org.apache.hudi.common.util.collection.Pair;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.exception.MetadataNotFoundException;
-import org.apache.hudi.index.HoodieIndex;
 import org.apache.hudi.index.HoodieIndexUtils;
+import org.apache.hudi.index.SparkHoodieIndex;
 import org.apache.hudi.io.HoodieRangeInfoHandle;
 import org.apache.hudi.table.HoodieTable;
 
@@ -38,7 +39,6 @@
 import org.apache.spark.Partitioner;
 import org.apache.spark.api.java.JavaPairRDD;
 import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
 import org.apache.spark.storage.StorageLevel;
 
 import java.util.ArrayList;
@@ -57,21 +57,22 @@
 /**
  * Indexing mechanism based on bloom filter. Each parquet file includes its row_key bloom filter in its metadata.
  */
-public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
+@SuppressWarnings("checkstyle:LineLength")
+public class SparkHoodieBloomIndex<T extends HoodieRecordPayload> extends SparkHoodieIndex<T> {
 
-  private static final Logger LOG = LogManager.getLogger(HoodieBloomIndex.class);
+  private static final Logger LOG = LogManager.getLogger(SparkHoodieBloomIndex.class);
 
-  public HoodieBloomIndex(HoodieWriteConfig config) {
+  public SparkHoodieBloomIndex(HoodieWriteConfig config) {
     super(config);
   }
 
   @Override
-  public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, JavaSparkContext jsc,
-                                              HoodieTable<T> hoodieTable) {
+  public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, HoodieEngineContext context,
+                                              HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) {
 
     // Step 0: cache the input record RDD
     if (config.getBloomIndexUseCaching()) {
-      recordRDD.persist(SparkConfigUtils.getBloomIndexInputStorageLevel(config.getProps()));
+      recordRDD.persist(SparkMemoryUtils.getBloomIndexInputStorageLevel(config.getProps()));
     }
 
     // Step 1: Extract out thinner JavaPairRDD of (partitionPath, recordKey)
@@ -80,7 +81,7 @@
 
     // Lookup indexes for all the partition/recordkey pair
     JavaPairRDD<HoodieKey, HoodieRecordLocation> keyFilenamePairRDD =
-        lookupIndex(partitionRecordKeyPairRDD, jsc, hoodieTable);
+        lookupIndex(partitionRecordKeyPairRDD, context, hoodieTable);
 
     // Cache the result, for subsequent stages.
     if (config.getBloomIndexUseCaching()) {
@@ -103,41 +104,11 @@
   }
 
   /**
-   * Returns an RDD mapping each HoodieKey with a partitionPath/fileID which contains it. Option.Empty if the key is not
-   * found.
-   *
-   * @param hoodieKeys  keys to lookup
-   * @param jsc         spark context
-   * @param hoodieTable hoodie table object
-   */
-  @Override
-  public JavaPairRDD<HoodieKey, Option<Pair<String, String>>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys,
-                                                                                  JavaSparkContext jsc, HoodieTable<T> hoodieTable) {
-    JavaPairRDD<String, String> partitionRecordKeyPairRDD =
-        hoodieKeys.mapToPair(key -> new Tuple2<>(key.getPartitionPath(), key.getRecordKey()));
-
-    // Lookup indexes for all the partition/recordkey pair
-    JavaPairRDD<HoodieKey, HoodieRecordLocation> recordKeyLocationRDD =
-        lookupIndex(partitionRecordKeyPairRDD, jsc, hoodieTable);
-    JavaPairRDD<HoodieKey, String> keyHoodieKeyPairRDD = hoodieKeys.mapToPair(key -> new Tuple2<>(key, null));
-
-    return keyHoodieKeyPairRDD.leftOuterJoin(recordKeyLocationRDD).mapToPair(keyLoc -> {
-      Option<Pair<String, String>> partitionPathFileidPair;
-      if (keyLoc._2._2.isPresent()) {
-        partitionPathFileidPair = Option.of(Pair.of(keyLoc._1().getPartitionPath(), keyLoc._2._2.get().getFileId()));
-      } else {
-        partitionPathFileidPair = Option.empty();
-      }
-      return new Tuple2<>(keyLoc._1, partitionPathFileidPair);
-    });
-  }
-
-  /**
    * Lookup the location for each record key and return the pair<record_key,location> for all record keys already
    * present and drop the record keys if not present.
    */
   private JavaPairRDD<HoodieKey, HoodieRecordLocation> lookupIndex(
-      JavaPairRDD<String, String> partitionRecordKeyPairRDD, final JavaSparkContext jsc,
+      JavaPairRDD<String, String> partitionRecordKeyPairRDD, final HoodieEngineContext context,
       final HoodieTable hoodieTable) {
     // Obtain records per partition, in the incoming records
     Map<String, Long> recordsPerPartition = partitionRecordKeyPairRDD.countByKey();
@@ -145,7 +116,7 @@
 
     // Step 2: Load all involved files as <Partition, filename> pairs
     List<Tuple2<String, BloomIndexFileInfo>> fileInfoList =
-        loadInvolvedFiles(affectedPartitionPathList, jsc, hoodieTable);
+        loadInvolvedFiles(affectedPartitionPathList, context, hoodieTable);
     final Map<String, List<BloomIndexFileInfo>> partitionToFileInfo =
         fileInfoList.stream().collect(groupingBy(Tuple2::_1, mapping(Tuple2::_2, toList())));
 
@@ -189,27 +160,27 @@
   /**
    * Load all involved files as <Partition, filename> pair RDD.
    */
-  List<Tuple2<String, BloomIndexFileInfo>> loadInvolvedFiles(List<String> partitions, final JavaSparkContext jsc,
+  List<Tuple2<String, BloomIndexFileInfo>> loadInvolvedFiles(List<String> partitions, final HoodieEngineContext context,
                                                              final HoodieTable hoodieTable) {
 
     // Obtain the latest data files from all the partitions.
-    List<Pair<String, String>> partitionPathFileIDList = getLatestBaseFilesForAllPartitions(partitions, jsc, hoodieTable).stream()
+    List<Pair<String, String>> partitionPathFileIDList = getLatestBaseFilesForAllPartitions(partitions, context, hoodieTable).stream()
         .map(pair -> Pair.of(pair.getKey(), pair.getValue().getFileId()))
         .collect(toList());
 
     if (config.getBloomIndexPruneByRanges()) {
       // also obtain file ranges, if range pruning is enabled
-      jsc.setJobDescription("Obtain key ranges for file slices (range pruning=on)");
-      return jsc.parallelize(partitionPathFileIDList, Math.max(partitionPathFileIDList.size(), 1)).mapToPair(pf -> {
+      context.setJobStatus(this.getClass().getName(), "Obtain key ranges for file slices (range pruning=on)");
+      return context.map(partitionPathFileIDList, pf -> {
         try {
-          HoodieRangeInfoHandle<T> rangeInfoHandle = new HoodieRangeInfoHandle<T>(config, hoodieTable, pf);
+          HoodieRangeInfoHandle rangeInfoHandle = new HoodieRangeInfoHandle(config, hoodieTable, pf);
           String[] minMaxKeys = rangeInfoHandle.getMinMaxKeys();
           return new Tuple2<>(pf.getKey(), new BloomIndexFileInfo(pf.getValue(), minMaxKeys[0], minMaxKeys[1]));
         } catch (MetadataNotFoundException me) {
           LOG.warn("Unable to find range metadata in file :" + pf);
           return new Tuple2<>(pf.getKey(), new BloomIndexFileInfo(pf.getValue()));
         }
-      }).collect();
+      }, Math.max(partitionPathFileIDList.size(), 1));
     } else {
       return partitionPathFileIDList.stream()
           .map(pf -> new Tuple2<>(pf.getKey(), new BloomIndexFileInfo(pf.getValue()))).collect(toList());
@@ -320,8 +291,8 @@
   }
 
   @Override
-  public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, JavaSparkContext jsc,
-                                             HoodieTable<T> hoodieTable) {
+  public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, HoodieEngineContext context,
+                                             HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) {
     return writeStatusRDD;
   }
 }
diff --git a/hudi-client/src/main/java/org/apache/hudi/index/bloom/HoodieGlobalBloomIndex.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieGlobalBloomIndex.java
similarity index 94%
rename from hudi-client/src/main/java/org/apache/hudi/index/bloom/HoodieGlobalBloomIndex.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieGlobalBloomIndex.java
index 4f93b30..771c01a 100644
--- a/hudi-client/src/main/java/org/apache/hudi/index/bloom/HoodieGlobalBloomIndex.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bloom/SparkHoodieGlobalBloomIndex.java
@@ -18,6 +18,7 @@
 
 package org.apache.hudi.index.bloom;
 
+import org.apache.hudi.client.common.HoodieEngineContext;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.EmptyHoodieRecordPayload;
 import org.apache.hudi.common.model.HoodieKey;
@@ -33,7 +34,6 @@
 
 import org.apache.spark.api.java.JavaPairRDD;
 import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
 import org.apache.spark.api.java.Optional;
 
 import java.io.IOException;
@@ -49,9 +49,9 @@
  * This filter will only work with hoodie table since it will only load partitions with .hoodie_partition_metadata
  * file in it.
  */
-public class HoodieGlobalBloomIndex<T extends HoodieRecordPayload> extends HoodieBloomIndex<T> {
+public class SparkHoodieGlobalBloomIndex<T extends HoodieRecordPayload> extends SparkHoodieBloomIndex<T> {
 
-  public HoodieGlobalBloomIndex(HoodieWriteConfig config) {
+  public SparkHoodieGlobalBloomIndex(HoodieWriteConfig config) {
     super(config);
   }
 
@@ -59,13 +59,13 @@
    * Load all involved files as <Partition, filename> pair RDD from all partitions in the table.
    */
   @Override
-  List<Tuple2<String, BloomIndexFileInfo>> loadInvolvedFiles(List<String> partitions, final JavaSparkContext jsc,
+  List<Tuple2<String, BloomIndexFileInfo>> loadInvolvedFiles(List<String> partitions, final HoodieEngineContext context,
                                                              final HoodieTable hoodieTable) {
     HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
     try {
       List<String> allPartitionPaths = FSUtils.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(),
           config.shouldAssumeDatePartitioning());
-      return super.loadInvolvedFiles(allPartitionPaths, jsc, hoodieTable);
+      return super.loadInvolvedFiles(allPartitionPaths, context, hoodieTable);
     } catch (IOException e) {
       throw new HoodieIOException("Failed to load all partitions", e);
     }
diff --git a/hudi-client/src/main/java/org/apache/hudi/index/hbase/HBaseIndex.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/hbase/SparkHoodieHBaseIndex.java
similarity index 94%
rename from hudi-client/src/main/java/org/apache/hudi/index/hbase/HBaseIndex.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/hbase/SparkHoodieHBaseIndex.java
index 4540f2f..21efd9b 100644
--- a/hudi-client/src/main/java/org/apache/hudi/index/hbase/HBaseIndex.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/hbase/SparkHoodieHBaseIndex.java
@@ -19,7 +19,9 @@
 package org.apache.hudi.index.hbase;
 
 import org.apache.hudi.client.WriteStatus;
-import org.apache.hudi.client.utils.SparkConfigUtils;
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
+import org.apache.hudi.client.utils.SparkMemoryUtils;
 import org.apache.hudi.common.model.HoodieKey;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieRecordLocation;
@@ -28,12 +30,11 @@
 import org.apache.hudi.common.table.timeline.HoodieTimeline;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.ReflectionUtils;
-import org.apache.hudi.common.util.collection.Pair;
 import org.apache.hudi.config.HoodieHBaseIndexConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.exception.HoodieDependentSystemUnavailableException;
 import org.apache.hudi.exception.HoodieIndexException;
-import org.apache.hudi.index.HoodieIndex;
+import org.apache.hudi.index.SparkHoodieIndex;
 import org.apache.hudi.table.HoodieTable;
 
 import org.apache.hadoop.conf.Configuration;
@@ -71,7 +72,7 @@
 /**
  * Hoodie Index implementation backed by HBase.
  */
-public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
+public class SparkHoodieHBaseIndex<T extends HoodieRecordPayload> extends SparkHoodieIndex<T> {
 
   public static final String DEFAULT_SPARK_EXECUTOR_INSTANCES_CONFIG_NAME = "spark.executor.instances";
   public static final String DEFAULT_SPARK_DYNAMIC_ALLOCATION_ENABLED_CONFIG_NAME = "spark.dynamicAllocation.enabled";
@@ -84,7 +85,7 @@
   private static final byte[] PARTITION_PATH_COLUMN = Bytes.toBytes("partition_path");
   private static final int SLEEP_TIME_MILLISECONDS = 100;
 
-  private static final Logger LOG = LogManager.getLogger(HBaseIndex.class);
+  private static final Logger LOG = LogManager.getLogger(SparkHoodieHBaseIndex.class);
   private static Connection hbaseConnection = null;
   private HBaseIndexQPSResourceAllocator hBaseIndexQPSResourceAllocator = null;
   private float qpsFraction;
@@ -98,7 +99,7 @@
   private final String tableName;
   private HBasePutBatchSizeCalculator putBatchSizeCalculator;
 
-  public HBaseIndex(HoodieWriteConfig config) {
+  public SparkHoodieHBaseIndex(HoodieWriteConfig config) {
     super(config);
     this.tableName = config.getHbaseTableName();
     addShutDownHook();
@@ -124,12 +125,6 @@
     return new DefaultHBaseQPSResourceAllocator(config);
   }
 
-  @Override
-  public JavaPairRDD<HoodieKey, Option<Pair<String, String>>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys,
-      JavaSparkContext jsc, HoodieTable<T> hoodieTable) {
-    throw new UnsupportedOperationException("HBase index does not implement check exist");
-  }
-
   private Connection getHBaseConnection() {
     Configuration hbaseConfig = HBaseConfiguration.create();
     String quorum = config.getHbaseZkQuorum();
@@ -195,7 +190,7 @@
       int multiGetBatchSize = config.getHbaseIndexGetBatchSize();
 
       // Grab the global HBase connection
-      synchronized (HBaseIndex.class) {
+      synchronized (SparkHoodieHBaseIndex.class) {
         if (hbaseConnection == null || hbaseConnection.isClosed()) {
           hbaseConnection = getHBaseConnection();
         }
@@ -255,8 +250,9 @@
   }
 
   @Override
-  public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, JavaSparkContext jsc,
-      HoodieTable<T> hoodieTable) {
+  public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
+                                              HoodieEngineContext context,
+                                              HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) {
     return recordRDD.mapPartitionsWithIndex(locationTagFunction(hoodieTable.getMetaClient()), true);
   }
 
@@ -266,7 +262,7 @@
 
       List<WriteStatus> writeStatusList = new ArrayList<>();
       // Grab the global HBase connection
-      synchronized (HBaseIndex.class) {
+      synchronized (SparkHoodieHBaseIndex.class) {
         if (hbaseConnection == null || hbaseConnection.isClosed()) {
           hbaseConnection = getHBaseConnection();
         }
@@ -339,20 +335,21 @@
   }
 
   @Override
-  public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, JavaSparkContext jsc,
-      HoodieTable<T> hoodieTable) {
+  public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, HoodieEngineContext context,
+                                             HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) {
     final HBaseIndexQPSResourceAllocator hBaseIndexQPSResourceAllocator = createQPSResourceAllocator(this.config);
-    setPutBatchSize(writeStatusRDD, hBaseIndexQPSResourceAllocator, jsc);
+    setPutBatchSize(writeStatusRDD, hBaseIndexQPSResourceAllocator, context);
     LOG.info("multiPutBatchSize: before hbase puts" + multiPutBatchSize);
     JavaRDD<WriteStatus> writeStatusJavaRDD = writeStatusRDD.mapPartitionsWithIndex(updateLocationFunction(), true);
     // caching the index updated status RDD
-    writeStatusJavaRDD = writeStatusJavaRDD.persist(SparkConfigUtils.getWriteStatusStorageLevel(config.getProps()));
+    writeStatusJavaRDD = writeStatusJavaRDD.persist(SparkMemoryUtils.getWriteStatusStorageLevel(config.getProps()));
     return writeStatusJavaRDD;
   }
 
   private void setPutBatchSize(JavaRDD<WriteStatus> writeStatusRDD,
-      HBaseIndexQPSResourceAllocator hBaseIndexQPSResourceAllocator, final JavaSparkContext jsc) {
+      HBaseIndexQPSResourceAllocator hBaseIndexQPSResourceAllocator, final HoodieEngineContext context) {
     if (config.getHbaseIndexPutBatchSizeAutoCompute()) {
+      JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context);
       SparkConf conf = jsc.getConf();
       int maxExecutors = conf.getInt(DEFAULT_SPARK_EXECUTOR_INSTANCES_CONFIG_NAME, 1);
       if (conf.getBoolean(DEFAULT_SPARK_DYNAMIC_ALLOCATION_ENABLED_CONFIG_NAME, false)) {
@@ -493,6 +490,6 @@
   }
 
   public void setHbaseConnection(Connection hbaseConnection) {
-    HBaseIndex.hbaseConnection = hbaseConnection;
+    SparkHoodieHBaseIndex.hbaseConnection = hbaseConnection;
   }
 }
diff --git a/hudi-client/src/main/java/org/apache/hudi/index/simple/HoodieGlobalSimpleIndex.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/simple/SparkHoodieGlobalSimpleIndex.java
similarity index 81%
rename from hudi-client/src/main/java/org/apache/hudi/index/simple/HoodieGlobalSimpleIndex.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/simple/SparkHoodieGlobalSimpleIndex.java
index 990f02d..bdb4991 100644
--- a/hudi-client/src/main/java/org/apache/hudi/index/simple/HoodieGlobalSimpleIndex.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/simple/SparkHoodieGlobalSimpleIndex.java
@@ -18,6 +18,8 @@
 
 package org.apache.hudi.index.simple;
 
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieEngineContext;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.EmptyHoodieRecordPayload;
 import org.apache.hudi.common.model.HoodieBaseFile;
@@ -34,7 +36,6 @@
 import org.apache.hudi.table.HoodieTable;
 import org.apache.spark.api.java.JavaPairRDD;
 import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
 import scala.Tuple2;
 
 import java.io.IOException;
@@ -50,31 +51,33 @@
  *
  * @param <T>
  */
-public class HoodieGlobalSimpleIndex<T extends HoodieRecordPayload> extends HoodieSimpleIndex<T> {
+@SuppressWarnings("checkstyle:LineLength")
+public class SparkHoodieGlobalSimpleIndex<T extends HoodieRecordPayload> extends SparkHoodieSimpleIndex<T> {
 
-  public HoodieGlobalSimpleIndex(HoodieWriteConfig config) {
+  public SparkHoodieGlobalSimpleIndex(HoodieWriteConfig config) {
     super(config);
   }
 
   @Override
-  public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, JavaSparkContext jsc,
-                                              HoodieTable<T> hoodieTable) {
-    return tagLocationInternal(recordRDD, jsc, hoodieTable);
+  public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, HoodieEngineContext context,
+                                              HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) {
+    return tagLocationInternal(recordRDD, context, hoodieTable);
   }
 
   /**
    * Tags records location for incoming records.
    *
    * @param inputRecordRDD   {@link JavaRDD} of incoming records
-   * @param jsc         instance of {@link JavaSparkContext} to use
+   * @param context         instance of {@link HoodieEngineContext} to use
    * @param hoodieTable instance of {@link HoodieTable} to use
    * @return {@link JavaRDD} of records with record locations set
    */
-  protected JavaRDD<HoodieRecord<T>> tagLocationInternal(JavaRDD<HoodieRecord<T>> inputRecordRDD, JavaSparkContext jsc,
-                                                         HoodieTable<T> hoodieTable) {
+  @Override
+  protected JavaRDD<HoodieRecord<T>> tagLocationInternal(JavaRDD<HoodieRecord<T>> inputRecordRDD, HoodieEngineContext context,
+                                                         HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) {
 
     JavaPairRDD<String, HoodieRecord<T>> keyedInputRecordRDD = inputRecordRDD.mapToPair(entry -> new Tuple2<>(entry.getRecordKey(), entry));
-    JavaPairRDD<HoodieKey, HoodieRecordLocation> allRecordLocationsInTable = fetchAllRecordLocations(jsc, hoodieTable,
+    JavaPairRDD<HoodieKey, HoodieRecordLocation> allRecordLocationsInTable = fetchAllRecordLocations(context, hoodieTable,
         config.getGlobalSimpleIndexParallelism());
     return getTaggedRecords(keyedInputRecordRDD, allRecordLocationsInTable);
   }
@@ -82,27 +85,28 @@
   /**
    * Fetch record locations for passed in {@link HoodieKey}s.
    *
-   * @param jsc         instance of {@link JavaSparkContext} to use
+   * @param context         instance of {@link HoodieEngineContext} to use
    * @param hoodieTable instance of {@link HoodieTable} of interest
    * @param parallelism parallelism to use
    * @return {@link JavaPairRDD} of {@link HoodieKey} and {@link HoodieRecordLocation}
    */
-  protected JavaPairRDD<HoodieKey, HoodieRecordLocation> fetchAllRecordLocations(JavaSparkContext jsc,
-                                                                                 HoodieTable hoodieTable,
+  protected JavaPairRDD<HoodieKey, HoodieRecordLocation> fetchAllRecordLocations(HoodieEngineContext context,
+                                                                                 HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable,
                                                                                  int parallelism) {
-    List<Pair<String, HoodieBaseFile>> latestBaseFiles = getAllBaseFilesInTable(jsc, hoodieTable);
-    return fetchRecordLocations(jsc, hoodieTable, parallelism, latestBaseFiles);
+    List<Pair<String, HoodieBaseFile>> latestBaseFiles = getAllBaseFilesInTable(context, hoodieTable);
+    return fetchRecordLocations(context, hoodieTable, parallelism, latestBaseFiles);
   }
 
   /**
    * Load all files for all partitions as <Partition, filename> pair RDD.
    */
-  protected List<Pair<String, HoodieBaseFile>> getAllBaseFilesInTable(final JavaSparkContext jsc, final HoodieTable hoodieTable) {
+  protected List<Pair<String, HoodieBaseFile>> getAllBaseFilesInTable(final HoodieEngineContext context,
+                                                                      final HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) {
     HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
     try {
       List<String> allPartitionPaths = FSUtils.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(), config.shouldAssumeDatePartitioning());
       // Obtain the latest data files from all the partitions.
-      return getLatestBaseFilesForAllPartitions(allPartitionPaths, jsc, hoodieTable);
+      return getLatestBaseFilesForAllPartitions(allPartitionPaths, context, hoodieTable);
     } catch (IOException e) {
       throw new HoodieIOException("Failed to load all partitions", e);
     }
@@ -149,21 +153,6 @@
         });
   }
 
-  /**
-   * Returns an RDD mapping each HoodieKey with a partitionPath/fileID which contains it. Option.Empty if the key is not.
-   * found.
-   *
-   * @param hoodieKeys  keys to lookup
-   * @param jsc         spark context
-   * @param hoodieTable hoodie table object
-   */
-  @Override
-  public JavaPairRDD<HoodieKey, Option<Pair<String, String>>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys,
-                                                                                  JavaSparkContext jsc,
-                                                                                  HoodieTable<T> hoodieTable) {
-    return fetchRecordLocationInternal(hoodieKeys, jsc, hoodieTable, config.getGlobalSimpleIndexParallelism());
-  }
-
   @Override
   public boolean isGlobal() {
     return true;
diff --git a/hudi-client/src/main/java/org/apache/hudi/index/simple/HoodieSimpleIndex.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/simple/SparkHoodieSimpleIndex.java
similarity index 61%
rename from hudi-client/src/main/java/org/apache/hudi/index/simple/HoodieSimpleIndex.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/simple/SparkHoodieSimpleIndex.java
index af963aa..3f167e2e 100644
--- a/hudi-client/src/main/java/org/apache/hudi/index/simple/HoodieSimpleIndex.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/simple/SparkHoodieSimpleIndex.java
@@ -19,7 +19,9 @@
 package org.apache.hudi.index.simple;
 
 import org.apache.hudi.client.WriteStatus;
-import org.apache.hudi.client.utils.SparkConfigUtils;
+import org.apache.hudi.client.utils.SparkMemoryUtils;
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.model.HoodieBaseFile;
 import org.apache.hudi.common.model.HoodieKey;
 import org.apache.hudi.common.model.HoodieRecord;
@@ -28,8 +30,8 @@
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.collection.Pair;
 import org.apache.hudi.config.HoodieWriteConfig;
-import org.apache.hudi.index.HoodieIndex;
 import org.apache.hudi.index.HoodieIndexUtils;
+import org.apache.hudi.index.SparkHoodieIndex;
 import org.apache.hudi.io.HoodieKeyLocationFetchHandle;
 import org.apache.hudi.table.HoodieTable;
 
@@ -49,15 +51,16 @@
  *
  * @param <T>
  */
-public class HoodieSimpleIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
+@SuppressWarnings("checkstyle:LineLength")
+public class SparkHoodieSimpleIndex<T extends HoodieRecordPayload> extends SparkHoodieIndex<T> {
 
-  public HoodieSimpleIndex(HoodieWriteConfig config) {
+  public SparkHoodieSimpleIndex(HoodieWriteConfig config) {
     super(config);
   }
 
   @Override
-  public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, JavaSparkContext jsc,
-                                             HoodieTable<T> hoodieTable) {
+  public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, HoodieEngineContext context,
+                                             HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) {
     return writeStatusRDD;
   }
 
@@ -82,42 +85,28 @@
   }
 
   @Override
-  public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, JavaSparkContext jsc,
-                                              HoodieTable<T> hoodieTable) {
-    return tagLocationInternal(recordRDD, jsc, hoodieTable);
-  }
-
-  /**
-   * Returns an RDD mapping each HoodieKey with a partitionPath/fileID which contains it. Option. Empty if the key is not
-   * found.
-   *
-   * @param hoodieKeys  keys to lookup
-   * @param jsc         spark context
-   * @param hoodieTable hoodie table object
-   */
-  @Override
-  public JavaPairRDD<HoodieKey, Option<Pair<String, String>>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys,
-                                                                                  JavaSparkContext jsc, HoodieTable<T> hoodieTable) {
-
-    return fetchRecordLocationInternal(hoodieKeys, jsc, hoodieTable, config.getSimpleIndexParallelism());
+  public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
+                                              HoodieEngineContext context,
+                                              HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) {
+    return tagLocationInternal(recordRDD, context, hoodieTable);
   }
 
   /**
    * Tags records location for incoming records.
    *
    * @param inputRecordRDD {@link JavaRDD} of incoming records
-   * @param jsc            instance of {@link JavaSparkContext} to use
+   * @param context            instance of {@link HoodieEngineContext} to use
    * @param hoodieTable    instance of {@link HoodieTable} to use
    * @return {@link JavaRDD} of records with record locations set
    */
-  protected JavaRDD<HoodieRecord<T>> tagLocationInternal(JavaRDD<HoodieRecord<T>> inputRecordRDD, JavaSparkContext jsc,
-                                                         HoodieTable<T> hoodieTable) {
+  protected JavaRDD<HoodieRecord<T>> tagLocationInternal(JavaRDD<HoodieRecord<T>> inputRecordRDD, HoodieEngineContext context,
+                                                         HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) {
     if (config.getSimpleIndexUseCaching()) {
-      inputRecordRDD.persist(SparkConfigUtils.getSimpleIndexInputStorageLevel(config.getProps()));
+      inputRecordRDD.persist(SparkMemoryUtils.getSimpleIndexInputStorageLevel(config.getProps()));
     }
 
     JavaPairRDD<HoodieKey, HoodieRecord<T>> keyedInputRecordRDD = inputRecordRDD.mapToPair(record -> new Tuple2<>(record.getKey(), record));
-    JavaPairRDD<HoodieKey, HoodieRecordLocation> existingLocationsOnTable = fetchRecordLocationsForAffectedPartitions(keyedInputRecordRDD.keys(), jsc, hoodieTable,
+    JavaPairRDD<HoodieKey, HoodieRecordLocation> existingLocationsOnTable = fetchRecordLocationsForAffectedPartitions(keyedInputRecordRDD.keys(), context, hoodieTable,
         config.getSimpleIndexParallelism());
 
     JavaRDD<HoodieRecord<T>> taggedRecordRDD = keyedInputRecordRDD.leftOuterJoin(existingLocationsOnTable)
@@ -134,46 +123,28 @@
   }
 
   /**
-   * Fetch record locations for passed in {@link JavaRDD} of HoodieKeys.
-   *
-   * @param lookupKeys  {@link JavaRDD} of {@link HoodieKey}s
-   * @param jsc         instance of {@link JavaSparkContext} to use
-   * @param hoodieTable instance of {@link HoodieTable} of interest
-   * @param parallelism parallelism to use
-   * @return Hoodiekeys mapped to partitionpath and filenames
-   */
-  JavaPairRDD<HoodieKey, Option<Pair<String, String>>> fetchRecordLocationInternal(JavaRDD<HoodieKey> lookupKeys, JavaSparkContext jsc,
-                                                                                   HoodieTable<T> hoodieTable, int parallelism) {
-    JavaPairRDD<HoodieKey, Option<HoodieRecordLocation>> keyLocationsRDD = lookupKeys.mapToPair(key -> new Tuple2<>(key, Option.empty()));
-    JavaPairRDD<HoodieKey, HoodieRecordLocation> existingRecords = fetchRecordLocationsForAffectedPartitions(lookupKeys, jsc, hoodieTable, parallelism);
-
-    return keyLocationsRDD.leftOuterJoin(existingRecords)
-        .mapToPair(entry -> {
-          final Option<HoodieRecordLocation> locationOpt = Option.ofNullable(entry._2._2.orNull());
-          final HoodieKey key = entry._1;
-          return locationOpt
-              .map(location -> new Tuple2<>(key, Option.of(Pair.of(key.getPartitionPath(), location.getFileId()))))
-              .orElse(new Tuple2<>(key, Option.empty()));
-        });
-  }
-
-  /**
    * Fetch record locations for passed in {@link HoodieKey}s.
    *
    * @param hoodieKeys  {@link JavaRDD} of {@link HoodieKey}s for which locations are fetched
-   * @param jsc         instance of {@link JavaSparkContext} to use
+   * @param context         instance of {@link HoodieEngineContext} to use
    * @param hoodieTable instance of {@link HoodieTable} of interest
    * @param parallelism parallelism to use
    * @return {@link JavaPairRDD} of {@link HoodieKey} and {@link HoodieRecordLocation}
    */
-  protected JavaPairRDD<HoodieKey, HoodieRecordLocation> fetchRecordLocationsForAffectedPartitions(JavaRDD<HoodieKey> hoodieKeys, JavaSparkContext jsc, HoodieTable<T> hoodieTable,
+  protected JavaPairRDD<HoodieKey, HoodieRecordLocation> fetchRecordLocationsForAffectedPartitions(JavaRDD<HoodieKey> hoodieKeys,
+                                                                                                   HoodieEngineContext context,
+                                                                                                   HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable,
                                                                                                    int parallelism) {
     List<String> affectedPartitionPathList = hoodieKeys.map(HoodieKey::getPartitionPath).distinct().collect();
-    List<Pair<String, HoodieBaseFile>> latestBaseFiles = getLatestBaseFilesForAllPartitions(affectedPartitionPathList, jsc, hoodieTable);
-    return fetchRecordLocations(jsc, hoodieTable, parallelism, latestBaseFiles);
+    List<Pair<String, HoodieBaseFile>> latestBaseFiles = getLatestBaseFilesForAllPartitions(affectedPartitionPathList, context, hoodieTable);
+    return fetchRecordLocations(context, hoodieTable, parallelism, latestBaseFiles);
   }
 
-  protected JavaPairRDD<HoodieKey, HoodieRecordLocation> fetchRecordLocations(JavaSparkContext jsc, HoodieTable<T> hoodieTable, int parallelism, List<Pair<String, HoodieBaseFile>> baseFiles) {
+  protected JavaPairRDD<HoodieKey, HoodieRecordLocation> fetchRecordLocations(HoodieEngineContext context,
+                                                                              HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable,
+                                                                              int parallelism,
+                                                                              List<Pair<String, HoodieBaseFile>> baseFiles) {
+    JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context);
     int fetchParallelism = Math.max(1, Math.max(baseFiles.size(), parallelism));
     return jsc.parallelize(baseFiles, fetchParallelism)
         .flatMapToPair(partitionPathBaseFile -> new HoodieKeyLocationFetchHandle(config, hoodieTable, partitionPathBaseFile).locations());
diff --git a/hudi-client/src/main/java/org/apache/hudi/io/HoodieRowCreateHandle.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/HoodieRowCreateHandle.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/io/HoodieRowCreateHandle.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/HoodieRowCreateHandle.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieInternalRowFileWriter.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieInternalRowFileWriter.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieInternalRowFileWriter.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieInternalRowFileWriter.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieInternalRowFileWriterFactory.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieInternalRowFileWriterFactory.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieInternalRowFileWriterFactory.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieInternalRowFileWriterFactory.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieInternalRowParquetWriter.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieInternalRowParquetWriter.java
similarity index 92%
rename from hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieInternalRowParquetWriter.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieInternalRowParquetWriter.java
index 7d05163..8070c07 100644
--- a/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieInternalRowParquetWriter.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieInternalRowParquetWriter.java
@@ -44,8 +44,8 @@
     super(HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf()),
         ParquetFileWriter.Mode.CREATE, parquetConfig.getWriteSupport(), parquetConfig.getCompressionCodecName(),
         parquetConfig.getBlockSize(), parquetConfig.getPageSize(), parquetConfig.getPageSize(),
-        ParquetWriter.DEFAULT_IS_DICTIONARY_ENABLED, ParquetWriter.DEFAULT_IS_VALIDATING_ENABLED,
-            ParquetWriter.DEFAULT_WRITER_VERSION, FSUtils.registerFileSystem(file, parquetConfig.getHadoopConf()));
+        DEFAULT_IS_DICTIONARY_ENABLED, DEFAULT_IS_VALIDATING_ENABLED,
+        DEFAULT_WRITER_VERSION, FSUtils.registerFileSystem(file, parquetConfig.getHadoopConf()));
     this.file = HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf());
     this.fs = (HoodieWrapperFileSystem) this.file.getFileSystem(FSUtils.registerFileSystem(file,
             parquetConfig.getHadoopConf()));
diff --git a/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieRowParquetConfig.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieRowParquetConfig.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieRowParquetConfig.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieRowParquetConfig.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieRowParquetWriteSupport.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieRowParquetWriteSupport.java
similarity index 100%
rename from hudi-client/src/main/java/org/apache/hudi/io/storage/HoodieRowParquetWriteSupport.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieRowParquetWriteSupport.java
diff --git a/hudi-client/src/main/java/org/apache/hudi/keygen/KeyGeneratorInterface.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/SparkKeyGeneratorInterface.java
similarity index 72%
copy from hudi-client/src/main/java/org/apache/hudi/keygen/KeyGeneratorInterface.java
copy to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/SparkKeyGeneratorInterface.java
index 6412a2f..77abf15 100644
--- a/hudi-client/src/main/java/org/apache/hudi/keygen/KeyGeneratorInterface.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/SparkKeyGeneratorInterface.java
@@ -18,24 +18,14 @@
 
 package org.apache.hudi.keygen;
 
-import org.apache.avro.generic.GenericRecord;
-import org.apache.hudi.common.model.HoodieKey;
 import org.apache.spark.sql.Row;
 
-import java.io.Serializable;
-import java.util.List;
-
 /**
- * Represents the interface key generators need to adhere to.
+ * Spark key generator interface.
  */
-public interface KeyGeneratorInterface extends Serializable {
-
-  HoodieKey getKey(GenericRecord record);
-
-  List<String> getRecordKeyFieldNames();
+public interface SparkKeyGeneratorInterface extends KeyGeneratorInterface {
 
   String getRecordKey(Row row);
 
   String getPartitionPath(Row row);
-
 }
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java
new file mode 100644
index 0000000..f2b3364
--- /dev/null
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.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.hudi.table;
+
+import org.apache.hudi.avro.model.HoodieCleanMetadata;
+import org.apache.hudi.avro.model.HoodieCompactionPlan;
+import org.apache.hudi.avro.model.HoodieRestoreMetadata;
+import org.apache.hudi.avro.model.HoodieRollbackMetadata;
+import org.apache.hudi.avro.model.HoodieSavepointMetadata;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
+import org.apache.hudi.common.model.HoodieBaseFile;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieNotSupportedException;
+import org.apache.hudi.exception.HoodieUpsertException;
+import org.apache.hudi.io.HoodieCreateHandle;
+import org.apache.hudi.io.HoodieMergeHandle;
+import org.apache.hudi.io.HoodieSortedMergeHandle;
+import org.apache.hudi.table.action.HoodieWriteMetadata;
+import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata;
+import org.apache.hudi.table.action.bootstrap.SparkBootstrapCommitActionExecutor;
+import org.apache.hudi.table.action.clean.SparkCleanActionExecutor;
+import org.apache.hudi.table.action.commit.SparkInsertOverwriteCommitActionExecutor;
+import org.apache.hudi.table.action.commit.SparkBulkInsertCommitActionExecutor;
+import org.apache.hudi.table.action.commit.SparkBulkInsertPreppedCommitActionExecutor;
+import org.apache.hudi.table.action.commit.SparkDeleteCommitActionExecutor;
+import org.apache.hudi.table.action.commit.SparkInsertCommitActionExecutor;
+import org.apache.hudi.table.action.commit.SparkInsertPreppedCommitActionExecutor;
+import org.apache.hudi.table.action.commit.SparkMergeHelper;
+import org.apache.hudi.table.action.commit.SparkUpsertCommitActionExecutor;
+import org.apache.hudi.table.action.commit.SparkUpsertPreppedCommitActionExecutor;
+import org.apache.hudi.table.action.restore.SparkCopyOnWriteRestoreActionExecutor;
+import org.apache.hudi.table.action.rollback.SparkCopyOnWriteRollbackActionExecutor;
+import org.apache.hudi.table.action.savepoint.SavepointActionExecutor;
+
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaRDD;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Implementation of a very heavily read-optimized Hoodie Table where, all data is stored in base files, with
+ * zero read amplification.
+ * <p>
+ * INSERTS - Produce new files, block aligned to desired size (or) Merge with the smallest existing file, to expand it
+ * <p>
+ * UPDATES - Produce a new version of the file, just replacing the updated records with new values
+ */
+public class HoodieSparkCopyOnWriteTable<T extends HoodieRecordPayload> extends HoodieSparkTable<T> {
+
+  private static final Logger LOG = LogManager.getLogger(HoodieSparkCopyOnWriteTable.class);
+
+  public HoodieSparkCopyOnWriteTable(HoodieWriteConfig config, HoodieEngineContext context, HoodieTableMetaClient metaClient) {
+    super(config, context, metaClient);
+  }
+
+  @Override
+  public HoodieWriteMetadata<JavaRDD<WriteStatus>> upsert(HoodieEngineContext context, String instantTime, JavaRDD<HoodieRecord<T>> records) {
+    return new SparkUpsertCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, records).execute();
+  }
+
+  @Override
+  public HoodieWriteMetadata<JavaRDD<WriteStatus>> insert(HoodieEngineContext context, String instantTime, JavaRDD<HoodieRecord<T>> records) {
+    return new SparkInsertCommitActionExecutor<>((HoodieSparkEngineContext)context, config, this, instantTime, records).execute();
+  }
+
+  @Override
+  public HoodieWriteMetadata<JavaRDD<WriteStatus>> bulkInsert(HoodieEngineContext context, String instantTime, JavaRDD<HoodieRecord<T>> records,
+      Option<BulkInsertPartitioner<JavaRDD<HoodieRecord<T>>>> userDefinedBulkInsertPartitioner) {
+    return new SparkBulkInsertCommitActionExecutor((HoodieSparkEngineContext) context, config,
+        this, instantTime, records, userDefinedBulkInsertPartitioner).execute();
+  }
+
+  @Override
+  public HoodieWriteMetadata<JavaRDD<WriteStatus>> delete(HoodieEngineContext context, String instantTime, JavaRDD<HoodieKey> keys) {
+    return new SparkDeleteCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, keys).execute();
+  }
+
+  @Override
+  public HoodieWriteMetadata<JavaRDD<WriteStatus>> upsertPrepped(HoodieEngineContext context, String instantTime,
+      JavaRDD<HoodieRecord<T>> preppedRecords) {
+    return new SparkUpsertPreppedCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, preppedRecords).execute();
+  }
+
+  @Override
+  public HoodieWriteMetadata<JavaRDD<WriteStatus>> insertPrepped(HoodieEngineContext context, String instantTime,
+      JavaRDD<HoodieRecord<T>> preppedRecords) {
+    return new SparkInsertPreppedCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, preppedRecords).execute();
+  }
+
+  @Override
+  public HoodieWriteMetadata<JavaRDD<WriteStatus>> bulkInsertPrepped(HoodieEngineContext context, String instantTime,
+      JavaRDD<HoodieRecord<T>> preppedRecords,  Option<BulkInsertPartitioner<JavaRDD<HoodieRecord<T>>>> userDefinedBulkInsertPartitioner) {
+    return new SparkBulkInsertPreppedCommitActionExecutor((HoodieSparkEngineContext) context, config,
+        this, instantTime, preppedRecords, userDefinedBulkInsertPartitioner).execute();
+  }
+
+  @Override
+  public HoodieWriteMetadata insertOverwrite(HoodieEngineContext context, String instantTime, JavaRDD<HoodieRecord<T>> records) {
+    return new SparkInsertOverwriteCommitActionExecutor(context, config, this, instantTime, records).execute();
+  }
+
+  @Override
+  public Option<HoodieCompactionPlan> scheduleCompaction(HoodieEngineContext context, String instantTime, Option<Map<String, String>> extraMetadata) {
+    throw new HoodieNotSupportedException("Compaction is not supported on a CopyOnWrite table");
+  }
+
+  @Override
+  public HoodieWriteMetadata<JavaRDD<WriteStatus>> compact(HoodieEngineContext context, String compactionInstantTime) {
+    throw new HoodieNotSupportedException("Compaction is not supported on a CopyOnWrite table");
+  }
+
+  @Override
+  public HoodieBootstrapWriteMetadata<JavaRDD<WriteStatus>> bootstrap(HoodieEngineContext context, Option<Map<String, String>> extraMetadata) {
+    return new SparkBootstrapCommitActionExecutor((HoodieSparkEngineContext) context, config, this, extraMetadata).execute();
+  }
+
+  @Override
+  public void rollbackBootstrap(HoodieEngineContext context, String instantTime) {
+    new SparkCopyOnWriteRestoreActionExecutor((HoodieSparkEngineContext) context, config, this, instantTime, HoodieTimeline.INIT_INSTANT_TS).execute();
+  }
+
+  public Iterator<List<WriteStatus>> handleUpdate(String instantTime, String partitionPath, String fileId,
+      Map<String, HoodieRecord<T>> keyToNewRecords, HoodieBaseFile oldDataFile) throws IOException {
+    // these are updates
+    HoodieMergeHandle upsertHandle = getUpdateHandle(instantTime, partitionPath, fileId, keyToNewRecords, oldDataFile);
+    return handleUpdateInternal(upsertHandle, instantTime, fileId);
+  }
+
+  protected Iterator<List<WriteStatus>> handleUpdateInternal(HoodieMergeHandle upsertHandle, String instantTime,
+      String fileId) throws IOException {
+    if (upsertHandle.getOldFilePath() == null) {
+      throw new HoodieUpsertException(
+          "Error in finding the old file path at commit " + instantTime + " for fileId: " + fileId);
+    } else {
+      SparkMergeHelper.newInstance().runMerge(this, upsertHandle);
+    }
+
+    // TODO(vc): This needs to be revisited
+    if (upsertHandle.getWriteStatus().getPartitionPath() == null) {
+      LOG.info("Upsert Handle has partition path as null " + upsertHandle.getOldFilePath() + ", "
+          + upsertHandle.getWriteStatus());
+    }
+    return Collections.singletonList(Collections.singletonList(upsertHandle.getWriteStatus())).iterator();
+  }
+
+  protected HoodieMergeHandle getUpdateHandle(String instantTime, String partitionPath, String fileId,
+      Map<String, HoodieRecord<T>> keyToNewRecords, HoodieBaseFile dataFileToBeMerged) {
+    if (requireSortedRecords()) {
+      return new HoodieSortedMergeHandle<>(config, instantTime, this, keyToNewRecords, partitionPath, fileId,
+          dataFileToBeMerged, taskContextSupplier);
+    } else {
+      return new HoodieMergeHandle<>(config, instantTime, this, keyToNewRecords, partitionPath, fileId,
+          dataFileToBeMerged,taskContextSupplier);
+    }
+  }
+
+  public Iterator<List<WriteStatus>> handleInsert(String instantTime, String partitionPath, String fileId,
+      Map<String, HoodieRecord<? extends HoodieRecordPayload>> recordMap) {
+    HoodieCreateHandle createHandle =
+        new HoodieCreateHandle(config, instantTime, this, partitionPath, fileId, recordMap, taskContextSupplier);
+    createHandle.write();
+    return Collections.singletonList(Collections.singletonList(createHandle.close())).iterator();
+  }
+
+  @Override
+  public HoodieCleanMetadata clean(HoodieEngineContext context, String cleanInstantTime) {
+    return new SparkCleanActionExecutor((HoodieSparkEngineContext)context, config, this, cleanInstantTime).execute();
+  }
+
+  @Override
+  public HoodieRollbackMetadata rollback(HoodieEngineContext context, String rollbackInstantTime, HoodieInstant commitInstant, boolean deleteInstants) {
+    return new SparkCopyOnWriteRollbackActionExecutor((HoodieSparkEngineContext) context, config, this, rollbackInstantTime, commitInstant, deleteInstants).execute();
+  }
+
+  @Override
+  public HoodieSavepointMetadata savepoint(HoodieEngineContext context, String instantToSavepoint, String user, String comment) {
+    return new SavepointActionExecutor(context, config, this, instantToSavepoint, user, comment).execute();
+  }
+
+  @Override
+  public HoodieRestoreMetadata restore(HoodieEngineContext context, String restoreInstantTime, String instantToRestore) {
+    return new SparkCopyOnWriteRestoreActionExecutor((HoodieSparkEngineContext) context, config, this, restoreInstantTime, instantToRestore).execute();
+  }
+
+}
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkMergeOnReadTable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkMergeOnReadTable.java
new file mode 100644
index 0000000..0a60dcc
--- /dev/null
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkMergeOnReadTable.java
@@ -0,0 +1,163 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.table;
+
+import org.apache.hudi.avro.model.HoodieCompactionPlan;
+import org.apache.hudi.avro.model.HoodieRestoreMetadata;
+import org.apache.hudi.avro.model.HoodieRollbackMetadata;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.model.HoodieWriteStat;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.table.action.HoodieWriteMetadata;
+import org.apache.hudi.table.action.bootstrap.SparkBootstrapDeltaCommitActionExecutor;
+import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata;
+import org.apache.hudi.table.action.compact.SparkRunCompactionActionExecutor;
+import org.apache.hudi.table.action.compact.SparkScheduleCompactionActionExecutor;
+import org.apache.hudi.table.action.deltacommit.SparkBulkInsertDeltaCommitActionExecutor;
+import org.apache.hudi.table.action.deltacommit.SparkBulkInsertPreppedDeltaCommitActionExecutor;
+import org.apache.hudi.table.action.deltacommit.SparkDeleteDeltaCommitActionExecutor;
+import org.apache.hudi.table.action.deltacommit.SparkInsertDeltaCommitActionExecutor;
+import org.apache.hudi.table.action.deltacommit.SparkInsertPreppedDeltaCommitActionExecutor;
+import org.apache.hudi.table.action.deltacommit.SparkUpsertDeltaCommitActionExecutor;
+import org.apache.hudi.table.action.deltacommit.SparkUpsertPreppedDeltaCommitActionExecutor;
+import org.apache.hudi.table.action.compact.BaseScheduleCompactionActionExecutor;
+import org.apache.hudi.table.action.restore.SparkMergeOnReadRestoreActionExecutor;
+import org.apache.hudi.table.action.rollback.SparkMergeOnReadRollbackActionExecutor;
+import org.apache.spark.api.java.JavaRDD;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Implementation of a more real-time Hoodie Table the provides tradeoffs on read and write cost/amplification.
+ *
+ * <p>
+ * INSERTS - Same as HoodieCopyOnWriteTable - Produce new files, block aligned to desired size (or) Merge with the
+ * smallest existing file, to expand it
+ * </p>
+ * <p>
+ * UPDATES - Appends the changes to a rolling log file maintained per file Id. Compaction merges the log file into the
+ * base file.
+ * </p>
+ * <p>
+ * WARNING - MOR table type does not support nested rollbacks, every rollback must be followed by an attempted commit
+ * action
+ * </p>
+ */
+public class HoodieSparkMergeOnReadTable<T extends HoodieRecordPayload> extends HoodieSparkCopyOnWriteTable<T> {
+
+  HoodieSparkMergeOnReadTable(HoodieWriteConfig config, HoodieEngineContext context, HoodieTableMetaClient metaClient) {
+    super(config, context, metaClient);
+  }
+
+  @Override
+  public HoodieWriteMetadata<JavaRDD<WriteStatus>> upsert(HoodieEngineContext context, String instantTime, JavaRDD<HoodieRecord<T>> records) {
+    return new SparkUpsertDeltaCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, records).execute();
+  }
+
+  @Override
+  public HoodieWriteMetadata<JavaRDD<WriteStatus>> insert(HoodieEngineContext context, String instantTime, JavaRDD<HoodieRecord<T>> records) {
+    return new SparkInsertDeltaCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, records).execute();
+  }
+
+  @Override
+  public HoodieWriteMetadata<JavaRDD<WriteStatus>> bulkInsert(HoodieEngineContext context, String instantTime, JavaRDD<HoodieRecord<T>> records,
+      Option<BulkInsertPartitioner<JavaRDD<HoodieRecord<T>>>> userDefinedBulkInsertPartitioner) {
+    return new SparkBulkInsertDeltaCommitActionExecutor((HoodieSparkEngineContext) context, config,
+        this, instantTime, records, userDefinedBulkInsertPartitioner).execute();
+  }
+
+  @Override
+  public HoodieWriteMetadata<JavaRDD<WriteStatus>> delete(HoodieEngineContext context, String instantTime, JavaRDD<HoodieKey> keys) {
+    return new SparkDeleteDeltaCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, keys).execute();
+  }
+
+  @Override
+  public HoodieWriteMetadata<JavaRDD<WriteStatus>> upsertPrepped(HoodieEngineContext context, String instantTime,
+      JavaRDD<HoodieRecord<T>> preppedRecords) {
+    return new SparkUpsertPreppedDeltaCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, preppedRecords).execute();
+  }
+
+  @Override
+  public HoodieWriteMetadata<JavaRDD<WriteStatus>> insertPrepped(HoodieEngineContext context, String instantTime,
+      JavaRDD<HoodieRecord<T>> preppedRecords) {
+    return new SparkInsertPreppedDeltaCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, preppedRecords).execute();
+  }
+
+  @Override
+  public HoodieWriteMetadata<JavaRDD<WriteStatus>> bulkInsertPrepped(HoodieEngineContext context, String instantTime,
+      JavaRDD<HoodieRecord<T>> preppedRecords,  Option<BulkInsertPartitioner<JavaRDD<HoodieRecord<T>>>> userDefinedBulkInsertPartitioner) {
+    return new SparkBulkInsertPreppedDeltaCommitActionExecutor((HoodieSparkEngineContext) context, config,
+        this, instantTime, preppedRecords, userDefinedBulkInsertPartitioner).execute();
+  }
+
+  @Override
+  public Option<HoodieCompactionPlan> scheduleCompaction(HoodieEngineContext context, String instantTime, Option<Map<String, String>> extraMetadata) {
+    BaseScheduleCompactionActionExecutor scheduleCompactionExecutor = new SparkScheduleCompactionActionExecutor(
+        context, config, this, instantTime, extraMetadata);
+    return scheduleCompactionExecutor.execute();
+  }
+
+  @Override
+  public HoodieWriteMetadata<JavaRDD<WriteStatus>> compact(HoodieEngineContext context, String compactionInstantTime) {
+    SparkRunCompactionActionExecutor compactionExecutor = new SparkRunCompactionActionExecutor((HoodieSparkEngineContext) context, config, this, compactionInstantTime);
+    return compactionExecutor.execute();
+  }
+
+  @Override
+  public HoodieBootstrapWriteMetadata<JavaRDD<WriteStatus>> bootstrap(HoodieEngineContext context, Option<Map<String, String>> extraMetadata) {
+    return new SparkBootstrapDeltaCommitActionExecutor((HoodieSparkEngineContext) context, config, this, extraMetadata).execute();
+  }
+
+  @Override
+  public void rollbackBootstrap(HoodieEngineContext context, String instantTime) {
+    new SparkMergeOnReadRestoreActionExecutor((HoodieSparkEngineContext) context, config, this, instantTime, HoodieTimeline.INIT_INSTANT_TS).execute();
+  }
+
+  @Override
+  public HoodieRollbackMetadata rollback(HoodieEngineContext context,
+                                         String rollbackInstantTime,
+                                         HoodieInstant commitInstant,
+                                         boolean deleteInstants) {
+    return new SparkMergeOnReadRollbackActionExecutor(context, config, this, rollbackInstantTime, commitInstant, deleteInstants).execute();
+  }
+
+  @Override
+  public HoodieRestoreMetadata restore(HoodieEngineContext context, String restoreInstantTime, String instantToRestore) {
+    return new SparkMergeOnReadRestoreActionExecutor((HoodieSparkEngineContext) context, config, this, restoreInstantTime, instantToRestore).execute();
+  }
+
+  @Override
+  public void finalizeWrite(HoodieEngineContext context, String instantTs, List<HoodieWriteStat> stats)
+      throws HoodieIOException {
+    // delegate to base class for MOR tables
+    super.finalizeWrite(context, instantTs, stats);
+  }
+}
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java
new file mode 100644
index 0000000..4292da7
--- /dev/null
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.table;
+
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.index.HoodieIndex;
+import org.apache.hudi.index.SparkHoodieIndex;
+
+import org.apache.spark.api.java.JavaRDD;
+
+public abstract class HoodieSparkTable<T extends HoodieRecordPayload>
+    extends HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
+
+  protected HoodieSparkTable(HoodieWriteConfig config, HoodieEngineContext context, HoodieTableMetaClient metaClient) {
+    super(config, context, metaClient);
+  }
+
+  public static <T extends HoodieRecordPayload> HoodieSparkTable<T> create(HoodieWriteConfig config, HoodieEngineContext context) {
+    HoodieTableMetaClient metaClient = new HoodieTableMetaClient(
+        context.getHadoopConf().get(),
+        config.getBasePath(),
+        true,
+        config.getConsistencyGuardConfig(),
+        Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion()))
+    );
+    return HoodieSparkTable.create(config, (HoodieSparkEngineContext) context, metaClient);
+  }
+
+  public static <T extends HoodieRecordPayload> HoodieSparkTable<T> create(HoodieWriteConfig config,
+                                                                           HoodieSparkEngineContext context,
+                                                                           HoodieTableMetaClient metaClient) {
+    switch (metaClient.getTableType()) {
+      case COPY_ON_WRITE:
+        return new HoodieSparkCopyOnWriteTable<>(config, context, metaClient);
+      case MERGE_ON_READ:
+        return new HoodieSparkMergeOnReadTable<>(config, context, metaClient);
+      default:
+        throw new HoodieException("Unsupported table type :" + metaClient.getTableType());
+    }
+  }
+
+  @Override
+  protected HoodieIndex<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> getIndex(HoodieWriteConfig config) {
+    return SparkHoodieIndex.createIndex(config);
+  }
+}
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java
similarity index 76%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapCommitActionExecutor.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java
index 47791d9..64df78c 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapCommitActionExecutor.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java
@@ -21,24 +21,29 @@
 import org.apache.hudi.avro.HoodieAvroUtils;
 import org.apache.hudi.avro.model.HoodieFileStatus;
 import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.bootstrap.HoodieBootstrapSchemaProvider;
 import org.apache.hudi.client.bootstrap.BootstrapMode;
 import org.apache.hudi.client.bootstrap.BootstrapRecordPayload;
-import org.apache.hudi.client.bootstrap.BootstrapSchemaProvider;
 import org.apache.hudi.client.bootstrap.BootstrapWriteStatus;
 import org.apache.hudi.client.bootstrap.FullRecordBootstrapDataProvider;
+import org.apache.hudi.client.bootstrap.HoodieSparkBootstrapSchemaProvider;
 import org.apache.hudi.client.bootstrap.selector.BootstrapModeSelector;
 import org.apache.hudi.client.bootstrap.translator.BootstrapPartitionPathTranslator;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
+import org.apache.hudi.client.utils.SparkMemoryUtils;
 import org.apache.hudi.common.bootstrap.FileStatusUtils;
 import org.apache.hudi.common.bootstrap.index.BootstrapIndex;
 import org.apache.hudi.common.config.TypedProperties;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.BootstrapFileMapping;
+import org.apache.hudi.common.model.HoodieCommitMetadata;
 import org.apache.hudi.common.model.HoodieKey;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.model.HoodieWriteStat;
 import org.apache.hudi.common.model.WriteOperationType;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
 import org.apache.hudi.common.table.timeline.HoodieInstant;
 import org.apache.hudi.common.table.timeline.HoodieInstant.State;
 import org.apache.hudi.common.table.timeline.HoodieTimeline;
@@ -49,16 +54,18 @@
 import org.apache.hudi.common.util.collection.Pair;
 import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor;
 import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieCommitException;
 import org.apache.hudi.exception.HoodieException;
 import org.apache.hudi.exception.HoodieIOException;
 import org.apache.hudi.execution.SparkBoundedInMemoryExecutor;
 import org.apache.hudi.io.HoodieBootstrapHandle;
 import org.apache.hudi.keygen.KeyGeneratorInterface;
+import org.apache.hudi.table.HoodieSparkTable;
 import org.apache.hudi.table.HoodieTable;
-import org.apache.hudi.table.WorkloadProfile;
 import org.apache.hudi.table.action.HoodieWriteMetadata;
+import org.apache.hudi.table.action.commit.BaseSparkCommitActionExecutor;
 import org.apache.hudi.table.action.commit.BaseCommitActionExecutor;
-import org.apache.hudi.table.action.commit.BulkInsertCommitActionExecutor;
+import org.apache.hudi.table.action.commit.SparkBulkInsertCommitActionExecutor;
 
 import org.apache.avro.Schema;
 import org.apache.avro.generic.GenericData;
@@ -66,7 +73,6 @@
 import org.apache.avro.generic.IndexedRecord;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hudi.table.action.commit.CommitActionExecutor;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
 import org.apache.parquet.avro.AvroParquetReader;
@@ -77,27 +83,31 @@
 import org.apache.parquet.hadoop.ParquetReader;
 import org.apache.parquet.hadoop.metadata.ParquetMetadata;
 import org.apache.parquet.schema.MessageType;
-import org.apache.spark.Partitioner;
 import org.apache.spark.api.java.JavaRDD;
 import org.apache.spark.api.java.JavaSparkContext;
 
 import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.time.Duration;
+import java.time.Instant;
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.stream.Collectors;
 
-public class BootstrapCommitActionExecutor<T extends HoodieRecordPayload<T>>
-    extends BaseCommitActionExecutor<T, HoodieBootstrapWriteMetadata> {
+public class SparkBootstrapCommitActionExecutor<T extends HoodieRecordPayload<T>>
+    extends BaseCommitActionExecutor<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>, HoodieBootstrapWriteMetadata> {
 
-  private static final Logger LOG = LogManager.getLogger(BootstrapCommitActionExecutor.class);
+  private static final Logger LOG = LogManager.getLogger(SparkBootstrapCommitActionExecutor.class);
   protected String bootstrapSchema = null;
   private transient FileSystem bootstrapSourceFileSystem;
 
-  public BootstrapCommitActionExecutor(JavaSparkContext jsc, HoodieWriteConfig config, HoodieTable<?> table,
-      Option<Map<String, String>> extraMetadata) {
-    super(jsc, new HoodieWriteConfig.Builder().withProps(config.getProps())
+  public SparkBootstrapCommitActionExecutor(HoodieSparkEngineContext context,
+                                            HoodieWriteConfig config,
+                                            HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table,
+                                            Option<Map<String, String>> extraMetadata) {
+    super(context, new HoodieWriteConfig.Builder().withProps(config.getProps())
         .withAutoCommit(true).withWriteStatusClass(BootstrapWriteStatus.class)
         .withBulkInsertParallelism(config.getBootstrapParallelism())
         .build(), table, HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS, WriteOperationType.BOOTSTRAP,
@@ -160,13 +170,32 @@
 
     JavaRDD<BootstrapWriteStatus> bootstrapWriteStatuses = runMetadataBootstrap(partitionFilesList);
 
-    HoodieWriteMetadata result = new HoodieWriteMetadata();
+    HoodieWriteMetadata<JavaRDD<WriteStatus>> result = new HoodieWriteMetadata<>();
     updateIndexAndCommitIfNeeded(bootstrapWriteStatuses.map(w -> w), result);
     return Option.of(result);
   }
 
+  private void updateIndexAndCommitIfNeeded(JavaRDD<WriteStatus> writeStatusRDD, HoodieWriteMetadata<JavaRDD<WriteStatus>> result) {
+    // cache writeStatusRDD before updating index, so that all actions before this are not triggered again for future
+    // RDD actions that are performed after updating the index.
+    writeStatusRDD = writeStatusRDD.persist(SparkMemoryUtils.getWriteStatusStorageLevel(config.getProps()));
+    Instant indexStartTime = Instant.now();
+    // Update the index back
+    JavaRDD<WriteStatus> statuses = table.getIndex().updateLocation(writeStatusRDD, context,
+        table);
+    result.setIndexUpdateDuration(Duration.between(indexStartTime, Instant.now()));
+    result.setWriteStatuses(statuses);
+    commitOnAutoCommit(result);
+  }
+
   @Override
-  protected void commit(Option<Map<String, String>> extraMetadata, HoodieWriteMetadata result) {
+  public HoodieWriteMetadata<JavaRDD<WriteStatus>> execute(JavaRDD<HoodieRecord<T>> inputRecords) {
+    // NO_OP
+    return null;
+  }
+
+  @Override
+  protected void commit(Option<Map<String, String>> extraMetadata, HoodieWriteMetadata<JavaRDD<WriteStatus>> result) {
     // Perform bootstrap index write and then commit. Make sure both record-key and bootstrap-index
     // is all done in a single job DAG.
     Map<String, List<Pair<BootstrapFileMapping, HoodieWriteStat>>> bootstrapSourceAndStats =
@@ -188,11 +217,45 @@
           + config.getBasePath());
     }
 
-    super.commit(extraMetadata, result, bootstrapSourceAndStats.values().stream()
+    commit(extraMetadata, result, bootstrapSourceAndStats.values().stream()
         .flatMap(f -> f.stream().map(Pair::getValue)).collect(Collectors.toList()));
     LOG.info("Committing metadata bootstrap !!");
   }
 
+  protected void commit(Option<Map<String, String>> extraMetadata, HoodieWriteMetadata<JavaRDD<WriteStatus>> result, List<HoodieWriteStat> stats) {
+    String actionType = table.getMetaClient().getCommitActionType();
+    LOG.info("Committing " + instantTime + ", action Type " + actionType);
+    // Create a Hoodie table which encapsulated the commits and files visible
+    HoodieSparkTable table = HoodieSparkTable.create(config, context);
+
+    HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
+    HoodieCommitMetadata metadata = new HoodieCommitMetadata();
+
+    result.setCommitted(true);
+    stats.forEach(stat -> metadata.addWriteStat(stat.getPartitionPath(), stat));
+    result.setWriteStats(stats);
+
+    // Finalize write
+    finalizeWrite(instantTime, stats, result);
+
+    // add in extra metadata
+    if (extraMetadata.isPresent()) {
+      extraMetadata.get().forEach(metadata::addMetadata);
+    }
+    metadata.addMetadata(HoodieCommitMetadata.SCHEMA_KEY, getSchemaToStoreInCommit());
+    metadata.setOperationType(operationType);
+
+    try {
+      activeTimeline.saveAsComplete(new HoodieInstant(true, actionType, instantTime),
+          Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
+      LOG.info("Committed " + instantTime);
+    } catch (IOException e) {
+      throw new HoodieCommitException("Failed to complete commit " + config.getBasePath() + " at time " + instantTime,
+          e);
+    }
+    result.setCommitMetadata(Option.of(metadata));
+  }
+
   /**
    * Perform Full Bootstrap.
    * @param partitionFilesList List of partitions and files within that partitions
@@ -205,9 +268,9 @@
     properties.putAll(config.getProps());
     FullRecordBootstrapDataProvider inputProvider =
         (FullRecordBootstrapDataProvider) ReflectionUtils.loadClass(config.getFullBootstrapInputProvider(),
-            properties, jsc);
+            properties, context);
     JavaRDD<HoodieRecord> inputRecordsRDD =
-        inputProvider.generateInputRecordRDD("bootstrap_source", config.getBootstrapSourceBasePath(),
+        (JavaRDD<HoodieRecord>) inputProvider.generateInputRecords("bootstrap_source", config.getBootstrapSourceBasePath(),
             partitionFilesList);
     // Start Full Bootstrap
     final HoodieInstant requested = new HoodieInstant(State.REQUESTED, table.getMetaClient().getCommitActionType(),
@@ -218,8 +281,8 @@
     return Option.of(getBulkInsertActionExecutor(inputRecordsRDD).execute());
   }
 
-  protected CommitActionExecutor<T> getBulkInsertActionExecutor(JavaRDD<HoodieRecord> inputRecordsRDD) {
-    return new BulkInsertCommitActionExecutor(jsc, new HoodieWriteConfig.Builder().withProps(config.getProps())
+  protected BaseSparkCommitActionExecutor<T> getBulkInsertActionExecutor(JavaRDD<HoodieRecord> inputRecordsRDD) {
+    return new SparkBulkInsertCommitActionExecutor((HoodieSparkEngineContext) context, new HoodieWriteConfig.Builder().withProps(config.getProps())
         .withSchema(bootstrapSchema).build(), table, HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS,
         inputRecordsRDD, extraMetadata);
   }
@@ -229,7 +292,7 @@
 
     Path sourceFilePath = FileStatusUtils.toPath(srcFileStatus.getPath());
     HoodieBootstrapHandle bootstrapHandle = new HoodieBootstrapHandle(config, HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS,
-        table, partitionPath, FSUtils.createNewFileIdPfx(), table.getSparkTaskContextSupplier());
+        table, partitionPath, FSUtils.createNewFileIdPfx(), table.getTaskContextSupplier());
     Schema avroSchema = null;
     try {
       ParquetMetadata readFooter = ParquetFileReader.readFooter(table.getHadoopConf(), sourceFilePath,
@@ -281,11 +344,11 @@
    */
   private Map<BootstrapMode, List<Pair<String, List<HoodieFileStatus>>>> listAndProcessSourcePartitions() throws IOException {
     List<Pair<String, List<HoodieFileStatus>>> folders = BootstrapUtils.getAllLeafFoldersWithFiles(
-            table.getMetaClient(), bootstrapSourceFileSystem, config.getBootstrapSourceBasePath(), jsc);
+            table.getMetaClient(), bootstrapSourceFileSystem, config.getBootstrapSourceBasePath(), context);
 
     LOG.info("Fetching Bootstrap Schema !!");
-    BootstrapSchemaProvider sourceSchemaProvider = new BootstrapSchemaProvider(config);
-    bootstrapSchema = sourceSchemaProvider.getBootstrapSchema(jsc, folders).toString();
+    HoodieBootstrapSchemaProvider sourceSchemaProvider = new HoodieSparkBootstrapSchemaProvider(config);
+    bootstrapSchema = sourceSchemaProvider.getBootstrapSchema(context, folders).toString();
     LOG.info("Bootstrap Schema :" + bootstrapSchema);
 
     BootstrapModeSelector selector =
@@ -305,6 +368,7 @@
   }
 
   private JavaRDD<BootstrapWriteStatus> runMetadataBootstrap(List<Pair<String, List<HoodieFileStatus>>> partitions) {
+    JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context);
     if (null == partitions || partitions.isEmpty()) {
       return jsc.emptyRDD();
     }
@@ -328,17 +392,6 @@
             partitionFsPair.getRight().getRight(), keyGenerator));
   }
 
-  //TODO: Once we decouple commit protocol, we should change the class hierarchy to avoid doing this.
-  @Override
-  protected Partitioner getUpsertPartitioner(WorkloadProfile profile) {
-    throw new UnsupportedOperationException("Should not called in bootstrap code path");
-  }
-
-  @Override
-  protected Partitioner getInsertPartitioner(WorkloadProfile profile) {
-    throw new UnsupportedOperationException("Should not called in bootstrap code path");
-  }
-
   @Override
   protected Iterator<List<WriteStatus>> handleInsert(String idPfx, Iterator<HoodieRecord<T>> recordItr) {
     throw new UnsupportedOperationException("Should not called in bootstrap code path");
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapDeltaCommitActionExecutor.java
new file mode 100644
index 0000000..59f8666
--- /dev/null
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapDeltaCommitActionExecutor.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.hudi.table.action.bootstrap;
+
+import java.util.Map;
+
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.table.action.commit.BaseSparkCommitActionExecutor;
+import org.apache.hudi.table.action.deltacommit.SparkBulkInsertDeltaCommitActionExecutor;
+import org.apache.spark.api.java.JavaRDD;
+
+public class SparkBootstrapDeltaCommitActionExecutor<T extends HoodieRecordPayload<T>>
+    extends SparkBootstrapCommitActionExecutor<T> {
+
+  public SparkBootstrapDeltaCommitActionExecutor(HoodieSparkEngineContext context,
+                                                 HoodieWriteConfig config, HoodieTable table,
+                                                 Option<Map<String, String>> extraMetadata) {
+    super(context, config, table, extraMetadata);
+  }
+
+  @Override
+  protected BaseSparkCommitActionExecutor<T> getBulkInsertActionExecutor(JavaRDD<HoodieRecord> inputRecordsRDD) {
+    return new SparkBulkInsertDeltaCommitActionExecutor((HoodieSparkEngineContext) context, new HoodieWriteConfig.Builder().withProps(config.getProps())
+        .withSchema(bootstrapSchema).build(), table, HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS,
+        inputRecordsRDD, extraMetadata);
+  }
+}
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/clean/SparkCleanActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/clean/SparkCleanActionExecutor.java
new file mode 100644
index 0000000..bbd5c1f
--- /dev/null
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/clean/SparkCleanActionExecutor.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.hudi.table.action.clean;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.avro.model.HoodieActionInstant;
+import org.apache.hudi.avro.model.HoodieCleanerPlan;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.HoodieCleanStat;
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
+import org.apache.hudi.common.model.CleanFileInfo;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.api.java.function.PairFlatMapFunction;
+import scala.Tuple2;
+
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+@SuppressWarnings("checkstyle:LineLength")
+public class SparkCleanActionExecutor<T extends HoodieRecordPayload> extends
+    BaseCleanActionExecutor<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
+
+  private static final Logger LOG = LogManager.getLogger(SparkCleanActionExecutor.class);
+
+  public SparkCleanActionExecutor(HoodieSparkEngineContext context,
+                                  HoodieWriteConfig config,
+                                  HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table,
+                                  String instantTime) {
+    super(context, config, table, instantTime);
+  }
+
+  private static PairFlatMapFunction<Iterator<Tuple2<String, CleanFileInfo>>, String, PartitionCleanStat>
+      deleteFilesFunc(HoodieTable table) {
+    return (PairFlatMapFunction<Iterator<Tuple2<String, CleanFileInfo>>, String, PartitionCleanStat>) iter -> {
+      Map<String, PartitionCleanStat> partitionCleanStatMap = new HashMap<>();
+      FileSystem fs = table.getMetaClient().getFs();
+      while (iter.hasNext()) {
+        Tuple2<String, CleanFileInfo> partitionDelFileTuple = iter.next();
+        String partitionPath = partitionDelFileTuple._1();
+        Path deletePath = new Path(partitionDelFileTuple._2().getFilePath());
+        String deletePathStr = deletePath.toString();
+        Boolean deletedFileResult = deleteFileAndGetResult(fs, deletePathStr);
+        if (!partitionCleanStatMap.containsKey(partitionPath)) {
+          partitionCleanStatMap.put(partitionPath, new PartitionCleanStat(partitionPath));
+        }
+        boolean isBootstrapBasePathFile = partitionDelFileTuple._2().isBootstrapBaseFile();
+        PartitionCleanStat partitionCleanStat = partitionCleanStatMap.get(partitionPath);
+        if (isBootstrapBasePathFile) {
+          // For Bootstrap Base file deletions, store the full file path.
+          partitionCleanStat.addDeleteFilePatterns(deletePath.toString(), true);
+          partitionCleanStat.addDeletedFileResult(deletePath.toString(), deletedFileResult, true);
+        } else {
+          partitionCleanStat.addDeleteFilePatterns(deletePath.getName(), false);
+          partitionCleanStat.addDeletedFileResult(deletePath.getName(), deletedFileResult, false);
+        }
+      }
+      return partitionCleanStatMap.entrySet().stream().map(e -> new Tuple2<>(e.getKey(), e.getValue()))
+          .collect(Collectors.toList()).iterator();
+    };
+  }
+
+  @Override
+  List<HoodieCleanStat> clean(HoodieEngineContext context, HoodieCleanerPlan cleanerPlan) {
+    JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context);
+    int cleanerParallelism = Math.min(
+        (int) (cleanerPlan.getFilePathsToBeDeletedPerPartition().values().stream().mapToInt(List::size).count()),
+        config.getCleanerParallelism());
+    LOG.info("Using cleanerParallelism: " + cleanerParallelism);
+
+    context.setJobStatus(this.getClass().getSimpleName(), "Perform cleaning of partitions");
+    List<Tuple2<String, PartitionCleanStat>> partitionCleanStats = jsc
+        .parallelize(cleanerPlan.getFilePathsToBeDeletedPerPartition().entrySet().stream()
+            .flatMap(x -> x.getValue().stream().map(y -> new Tuple2<>(x.getKey(),
+                new CleanFileInfo(y.getFilePath(), y.getIsBootstrapBaseFile()))))
+            .collect(Collectors.toList()), cleanerParallelism)
+        .mapPartitionsToPair(deleteFilesFunc(table))
+        .reduceByKey(PartitionCleanStat::merge).collect();
+
+    Map<String, PartitionCleanStat> partitionCleanStatsMap = partitionCleanStats.stream()
+        .collect(Collectors.toMap(Tuple2::_1, Tuple2::_2));
+
+    // Return PartitionCleanStat for each partition passed.
+    return cleanerPlan.getFilePathsToBeDeletedPerPartition().keySet().stream().map(partitionPath -> {
+      PartitionCleanStat partitionCleanStat = partitionCleanStatsMap.containsKey(partitionPath)
+          ? partitionCleanStatsMap.get(partitionPath)
+          : new PartitionCleanStat(partitionPath);
+      HoodieActionInstant actionInstant = cleanerPlan.getEarliestInstantToRetain();
+      return HoodieCleanStat.newBuilder().withPolicy(config.getCleanerPolicy()).withPartitionPath(partitionPath)
+          .withEarliestCommitRetained(Option.ofNullable(
+              actionInstant != null
+                  ? new HoodieInstant(HoodieInstant.State.valueOf(actionInstant.getState()),
+                  actionInstant.getAction(), actionInstant.getTimestamp())
+                  : null))
+          .withDeletePathPattern(partitionCleanStat.deletePathPatterns())
+          .withSuccessfulDeletes(partitionCleanStat.successDeleteFiles())
+          .withFailedDeletes(partitionCleanStat.failedDeleteFiles())
+          .withDeleteBootstrapBasePathPatterns(partitionCleanStat.getDeleteBootstrapBasePathPatterns())
+          .withSuccessfulDeleteBootstrapBaseFiles(partitionCleanStat.getSuccessfulDeleteBootstrapBaseFiles())
+          .withFailedDeleteBootstrapBaseFiles(partitionCleanStat.getFailedDeleteBootstrapBaseFiles())
+          .build();
+    }).collect(Collectors.toList());
+  }
+}
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java
new file mode 100644
index 0000000..36cca8c
--- /dev/null
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java
@@ -0,0 +1,342 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.table.action.commit;
+
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.utils.SparkMemoryUtils;
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.common.model.HoodieBaseFile;
+import org.apache.hudi.common.model.HoodieCommitMetadata;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordLocation;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.model.HoodieWriteStat;
+import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.CommitUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieCommitException;
+import org.apache.hudi.exception.HoodieUpsertException;
+import org.apache.hudi.execution.SparkLazyInsertIterable;
+import org.apache.hudi.io.CreateHandleFactory;
+import org.apache.hudi.io.HoodieMergeHandle;
+import org.apache.hudi.io.HoodieSortedMergeHandle;
+import org.apache.hudi.table.HoodieSparkTable;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.table.WorkloadProfile;
+import org.apache.hudi.table.WorkloadStat;
+import org.apache.hudi.table.action.HoodieWriteMetadata;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.Partitioner;
+import org.apache.spark.api.java.JavaPairRDD;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.storage.StorageLevel;
+import scala.Tuple2;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.nio.charset.StandardCharsets;
+import java.time.Duration;
+import java.time.Instant;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+public abstract class BaseSparkCommitActionExecutor<T extends HoodieRecordPayload> extends
+    BaseCommitActionExecutor<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>, HoodieWriteMetadata> {
+
+  private static final Logger LOG = LogManager.getLogger(BaseSparkCommitActionExecutor.class);
+
+  public BaseSparkCommitActionExecutor(HoodieEngineContext context,
+                                       HoodieWriteConfig config,
+                                       HoodieTable table,
+                                       String instantTime,
+                                       WriteOperationType operationType) {
+    super(context, config, table, instantTime, operationType, Option.empty());
+  }
+
+  public BaseSparkCommitActionExecutor(HoodieEngineContext context,
+                                       HoodieWriteConfig config,
+                                       HoodieTable table,
+                                       String instantTime,
+                                       WriteOperationType operationType,
+                                       Option extraMetadata) {
+    super(context, config, table, instantTime, operationType, extraMetadata);
+  }
+
+  @Override
+  public HoodieWriteMetadata<JavaRDD<WriteStatus>> execute(JavaRDD<HoodieRecord<T>> inputRecordsRDD) {
+    HoodieWriteMetadata<JavaRDD<WriteStatus>> result = new HoodieWriteMetadata<>();
+    // Cache the tagged records, so we don't end up computing both
+    // TODO: Consistent contract in HoodieWriteClient regarding preppedRecord storage level handling
+    if (inputRecordsRDD.getStorageLevel() == StorageLevel.NONE()) {
+      inputRecordsRDD.persist(StorageLevel.MEMORY_AND_DISK_SER());
+    } else {
+      LOG.info("RDD PreppedRecords was persisted at: " + inputRecordsRDD.getStorageLevel());
+    }
+
+    WorkloadProfile profile = null;
+    if (isWorkloadProfileNeeded()) {
+      profile = new WorkloadProfile(buildProfile(inputRecordsRDD));
+      LOG.info("Workload profile :" + profile);
+      saveWorkloadProfileMetadataToInflight(profile, instantTime);
+    }
+
+    // partition using the insert partitioner
+    final Partitioner partitioner = getPartitioner(profile);
+    JavaRDD<HoodieRecord<T>> partitionedRecords = partition(inputRecordsRDD, partitioner);
+    JavaRDD<WriteStatus> writeStatusRDD = partitionedRecords.mapPartitionsWithIndex((partition, recordItr) -> {
+      if (WriteOperationType.isChangingRecords(operationType)) {
+        return handleUpsertPartition(instantTime, partition, recordItr, partitioner);
+      } else {
+        return handleInsertPartition(instantTime, partition, recordItr, partitioner);
+      }
+    }, true).flatMap(List::iterator);
+
+    updateIndexAndCommitIfNeeded(writeStatusRDD, result);
+    return result;
+  }
+
+  private Pair<HashMap<String, WorkloadStat>, WorkloadStat> buildProfile(JavaRDD<HoodieRecord<T>> inputRecordsRDD) {
+    HashMap<String, WorkloadStat> partitionPathStatMap = new HashMap<>();
+    WorkloadStat globalStat = new WorkloadStat();
+
+    // group the records by partitionPath + currentLocation combination, count the number of
+    // records in each partition
+    Map<Tuple2<String, Option<HoodieRecordLocation>>, Long> partitionLocationCounts = inputRecordsRDD
+        .mapToPair(record -> new Tuple2<>(
+            new Tuple2<>(record.getPartitionPath(), Option.ofNullable(record.getCurrentLocation())), record))
+        .countByKey();
+
+    // count the number of both inserts and updates in each partition, update the counts to workLoadStats
+    for (Map.Entry<Tuple2<String, Option<HoodieRecordLocation>>, Long> e : partitionLocationCounts.entrySet()) {
+      String partitionPath = e.getKey()._1();
+      Long count = e.getValue();
+      Option<HoodieRecordLocation> locOption = e.getKey()._2();
+
+      if (!partitionPathStatMap.containsKey(partitionPath)) {
+        partitionPathStatMap.put(partitionPath, new WorkloadStat());
+      }
+
+      if (locOption.isPresent()) {
+        // update
+        partitionPathStatMap.get(partitionPath).addUpdates(locOption.get(), count);
+        globalStat.addUpdates(locOption.get(), count);
+      } else {
+        // insert
+        partitionPathStatMap.get(partitionPath).addInserts(count);
+        globalStat.addInserts(count);
+      }
+    }
+    return Pair.of(partitionPathStatMap, globalStat);
+  }
+
+  protected Partitioner getPartitioner(WorkloadProfile profile) {
+    if (WriteOperationType.isChangingRecords(operationType)) {
+      return getUpsertPartitioner(profile);
+    } else {
+      return getInsertPartitioner(profile);
+    }
+  }
+
+  private JavaRDD<HoodieRecord<T>> partition(JavaRDD<HoodieRecord<T>> dedupedRecords, Partitioner partitioner) {
+    JavaPairRDD<Tuple2, HoodieRecord<T>> mappedRDD = dedupedRecords.mapToPair(
+        record -> new Tuple2<>(new Tuple2<>(record.getKey(), Option.ofNullable(record.getCurrentLocation())), record));
+
+    JavaPairRDD<Tuple2, HoodieRecord<T>> partitionedRDD;
+    if (table.requireSortedRecords()) {
+      // Partition and sort within each partition as a single step. This is faster than partitioning first and then
+      // applying a sort.
+      Comparator<Tuple2> comparator = (Comparator<Tuple2> & Serializable)(t1, t2) -> {
+        HoodieKey key1 = (HoodieKey) t1._1;
+        HoodieKey key2 = (HoodieKey) t2._1;
+        return key1.getRecordKey().compareTo(key2.getRecordKey());
+      };
+
+      partitionedRDD = mappedRDD.repartitionAndSortWithinPartitions(partitioner, comparator);
+    } else {
+      // Partition only
+      partitionedRDD = mappedRDD.partitionBy(partitioner);
+    }
+
+    return partitionedRDD.map(Tuple2::_2);
+  }
+
+  protected void updateIndexAndCommitIfNeeded(JavaRDD<WriteStatus> writeStatusRDD, HoodieWriteMetadata result) {
+    // cache writeStatusRDD before updating index, so that all actions before this are not triggered again for future
+    // RDD actions that are performed after updating the index.
+    writeStatusRDD = writeStatusRDD.persist(SparkMemoryUtils.getWriteStatusStorageLevel(config.getProps()));
+    Instant indexStartTime = Instant.now();
+    // Update the index back
+    JavaRDD<WriteStatus> statuses = table.getIndex().updateLocation(writeStatusRDD, context, table);
+    result.setIndexUpdateDuration(Duration.between(indexStartTime, Instant.now()));
+    result.setWriteStatuses(statuses);
+    result.setPartitionToReplaceFileIds(getPartitionToReplacedFileIds(statuses));
+    commitOnAutoCommit(result);
+  }
+
+  protected String getCommitActionType() {
+    return  table.getMetaClient().getCommitActionType();
+  }
+
+  @Override
+  protected void commit(Option<Map<String, String>> extraMetadata, HoodieWriteMetadata<JavaRDD<WriteStatus>> result) {
+    commit(extraMetadata, result, result.getWriteStatuses().map(WriteStatus::getStat).collect());
+  }
+
+  protected void commit(Option<Map<String, String>> extraMetadata, HoodieWriteMetadata<JavaRDD<WriteStatus>> result, List<HoodieWriteStat> writeStats) {
+    String actionType = getCommitActionType();
+    LOG.info("Committing " + instantTime + ", action Type " + actionType);
+    result.setCommitted(true);
+    result.setWriteStats(writeStats);
+    // Finalize write
+    finalizeWrite(instantTime, writeStats, result);
+
+    try {
+      LOG.info("Committing " + instantTime + ", action Type " + getCommitActionType());
+      HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
+      HoodieCommitMetadata metadata = CommitUtils.buildMetadata(writeStats, result.getPartitionToReplaceFileIds(),
+          extraMetadata, operationType, getSchemaToStoreInCommit(), getCommitActionType());
+
+      activeTimeline.saveAsComplete(new HoodieInstant(true, getCommitActionType(), instantTime),
+          Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
+      LOG.info("Committed " + instantTime);
+      result.setCommitMetadata(Option.of(metadata));
+    } catch (IOException e) {
+      throw new HoodieCommitException("Failed to complete commit " + config.getBasePath() + " at time " + instantTime,
+          e);
+    }
+  }
+
+  protected Map<String, List<String>> getPartitionToReplacedFileIds(JavaRDD<WriteStatus> writeStatuses) {
+    return Collections.emptyMap();
+  }
+
+  @SuppressWarnings("unchecked")
+  protected Iterator<List<WriteStatus>> handleUpsertPartition(String instantTime, Integer partition, Iterator recordItr,
+                                                              Partitioner partitioner) {
+    UpsertPartitioner upsertPartitioner = (UpsertPartitioner) partitioner;
+    BucketInfo binfo = upsertPartitioner.getBucketInfo(partition);
+    BucketType btype = binfo.bucketType;
+    try {
+      if (btype.equals(BucketType.INSERT)) {
+        return handleInsert(binfo.fileIdPrefix, recordItr);
+      } else if (btype.equals(BucketType.UPDATE)) {
+        return handleUpdate(binfo.partitionPath, binfo.fileIdPrefix, recordItr);
+      } else {
+        throw new HoodieUpsertException("Unknown bucketType " + btype + " for partition :" + partition);
+      }
+    } catch (Throwable t) {
+      String msg = "Error upserting bucketType " + btype + " for partition :" + partition;
+      LOG.error(msg, t);
+      throw new HoodieUpsertException(msg, t);
+    }
+  }
+
+  protected Iterator<List<WriteStatus>> handleInsertPartition(String instantTime, Integer partition, Iterator recordItr,
+                                                              Partitioner partitioner) {
+    return handleUpsertPartition(instantTime, partition, recordItr, partitioner);
+  }
+
+  @Override
+  public Iterator<List<WriteStatus>> handleUpdate(String partitionPath, String fileId,
+                                                  Iterator<HoodieRecord<T>> recordItr)
+      throws IOException {
+    // This is needed since sometimes some buckets are never picked in getPartition() and end up with 0 records
+    if (!recordItr.hasNext()) {
+      LOG.info("Empty partition with fileId => " + fileId);
+      return Collections.singletonList((List<WriteStatus>) Collections.EMPTY_LIST).iterator();
+    }
+    // these are updates
+    HoodieMergeHandle upsertHandle = getUpdateHandle(partitionPath, fileId, recordItr);
+    return handleUpdateInternal(upsertHandle, fileId);
+  }
+
+  public Iterator<List<WriteStatus>> handleUpdate(String partitionPath, String fileId,
+                                                  Map<String, HoodieRecord<T>> keyToNewRecords,
+                                                  HoodieBaseFile oldDataFile) throws IOException {
+    // these are updates
+    HoodieMergeHandle upsertHandle = getUpdateHandle(partitionPath, fileId, keyToNewRecords, oldDataFile);
+    return handleUpdateInternal(upsertHandle, fileId);
+  }
+
+  protected Iterator<List<WriteStatus>> handleUpdateInternal(HoodieMergeHandle upsertHandle, String fileId)
+      throws IOException {
+    if (upsertHandle.getOldFilePath() == null) {
+      throw new HoodieUpsertException(
+          "Error in finding the old file path at commit " + instantTime + " for fileId: " + fileId);
+    } else {
+      SparkMergeHelper.newInstance().runMerge(table, upsertHandle);
+    }
+
+    // TODO(vc): This needs to be revisited
+    if (upsertHandle.getWriteStatus().getPartitionPath() == null) {
+      LOG.info("Upsert Handle has partition path as null " + upsertHandle.getOldFilePath() + ", "
+          + upsertHandle.getWriteStatus());
+    }
+    return Collections.singletonList(Collections.singletonList(upsertHandle.getWriteStatus())).iterator();
+  }
+
+  protected HoodieMergeHandle getUpdateHandle(String partitionPath, String fileId, Iterator<HoodieRecord<T>> recordItr) {
+    if (table.requireSortedRecords()) {
+      return new HoodieSortedMergeHandle<>(config, instantTime, (HoodieSparkTable) table, recordItr, partitionPath, fileId, taskContextSupplier);
+    } else {
+      return new HoodieMergeHandle<>(config, instantTime, table, recordItr, partitionPath, fileId, taskContextSupplier);
+    }
+  }
+
+  protected HoodieMergeHandle getUpdateHandle(String partitionPath, String fileId,
+                                              Map<String, HoodieRecord<T>> keyToNewRecords,
+                                              HoodieBaseFile dataFileToBeMerged) {
+    return new HoodieMergeHandle<>(config, instantTime, table, keyToNewRecords,
+        partitionPath, fileId, dataFileToBeMerged, taskContextSupplier);
+  }
+
+  @Override
+  public Iterator<List<WriteStatus>> handleInsert(String idPfx, Iterator<HoodieRecord<T>> recordItr)
+      throws Exception {
+    // This is needed since sometimes some buckets are never picked in getPartition() and end up with 0 records
+    if (!recordItr.hasNext()) {
+      LOG.info("Empty partition");
+      return Collections.singletonList((List<WriteStatus>) Collections.EMPTY_LIST).iterator();
+    }
+    return new SparkLazyInsertIterable(recordItr, true, config, instantTime, table, idPfx,
+        taskContextSupplier, new CreateHandleFactory<>());
+  }
+
+  public Partitioner getUpsertPartitioner(WorkloadProfile profile) {
+    if (profile == null) {
+      throw new HoodieUpsertException("Need workload profile to construct the upsert partitioner.");
+    }
+    return new UpsertPartitioner(profile, context, table, config);
+  }
+
+  public Partitioner getInsertPartitioner(WorkloadProfile profile) {
+    return getUpsertPartitioner(profile);
+  }
+
+}
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/BulkInsertCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertCommitActionExecutor.java
similarity index 67%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/commit/BulkInsertCommitActionExecutor.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertCommitActionExecutor.java
index 162ae29..fb8b5f9 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/BulkInsertCommitActionExecutor.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertCommitActionExecutor.java
@@ -19,6 +19,9 @@
 package org.apache.hudi.table.action.commit;
 
 import java.util.Map;
+
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.model.WriteOperationType;
@@ -30,32 +33,31 @@
 
 import org.apache.hudi.table.action.HoodieWriteMetadata;
 import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
 
-public class BulkInsertCommitActionExecutor<T extends HoodieRecordPayload<T>> extends CommitActionExecutor<T> {
+public class SparkBulkInsertCommitActionExecutor<T extends HoodieRecordPayload<T>> extends BaseSparkCommitActionExecutor<T> {
 
   private final JavaRDD<HoodieRecord<T>> inputRecordsRDD;
   private final Option<BulkInsertPartitioner<T>> bulkInsertPartitioner;
 
-  public BulkInsertCommitActionExecutor(JavaSparkContext jsc, HoodieWriteConfig config, HoodieTable table,
-                                        String instantTime, JavaRDD<HoodieRecord<T>> inputRecordsRDD,
-                                        Option<BulkInsertPartitioner<T>> bulkInsertPartitioner) {
-    this(jsc, config, table, instantTime, inputRecordsRDD, bulkInsertPartitioner, Option.empty());
+  public SparkBulkInsertCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table,
+                                             String instantTime, JavaRDD<HoodieRecord<T>> inputRecordsRDD,
+                                             Option<BulkInsertPartitioner<T>> bulkInsertPartitioner) {
+    this(context, config, table, instantTime, inputRecordsRDD, bulkInsertPartitioner, Option.empty());
   }
 
-  public BulkInsertCommitActionExecutor(JavaSparkContext jsc, HoodieWriteConfig config, HoodieTable table,
+  public SparkBulkInsertCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table,
                                         String instantTime, JavaRDD<HoodieRecord<T>> inputRecordsRDD,
                                         Option<BulkInsertPartitioner<T>> bulkInsertPartitioner,
                                         Option<Map<String, String>> extraMetadata) {
-    super(jsc, config, table, instantTime, WriteOperationType.BULK_INSERT, extraMetadata);
+    super(context, config, table, instantTime, WriteOperationType.BULK_INSERT, extraMetadata);
     this.inputRecordsRDD = inputRecordsRDD;
     this.bulkInsertPartitioner = bulkInsertPartitioner;
   }
 
   @Override
-  public HoodieWriteMetadata execute() {
+  public HoodieWriteMetadata<JavaRDD<WriteStatus>> execute() {
     try {
-      return BulkInsertHelper.bulkInsert(inputRecordsRDD, instantTime, (HoodieTable<T>) table, config,
+      return SparkBulkInsertHelper.newInstance().bulkInsert(inputRecordsRDD, instantTime, table, config,
           this, true, bulkInsertPartitioner);
     } catch (HoodieInsertException ie) {
       throw ie;
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertHelper.java
new file mode 100644
index 0000000..9ccd66b
--- /dev/null
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertHelper.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.hudi.table.action.commit;
+
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.execution.bulkinsert.BulkInsertInternalPartitionerFactory;
+import org.apache.hudi.execution.bulkinsert.BulkInsertMapFunction;
+import org.apache.hudi.table.BulkInsertPartitioner;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.table.action.HoodieWriteMetadata;
+
+import org.apache.spark.api.java.JavaRDD;
+
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * A spark implementation of {@link AbstractBulkInsertHelper}.
+ *
+ * @param <T>
+ */
+@SuppressWarnings("checkstyle:LineLength")
+public class SparkBulkInsertHelper<T extends HoodieRecordPayload, R> extends AbstractBulkInsertHelper<T, JavaRDD<HoodieRecord<T>>,
+    JavaRDD<HoodieKey>, JavaRDD<WriteStatus>, R> {
+
+  private SparkBulkInsertHelper() {
+  }
+
+  private static class BulkInsertHelperHolder {
+    private static final SparkBulkInsertHelper SPARK_BULK_INSERT_HELPER = new SparkBulkInsertHelper();
+  }
+
+  public static SparkBulkInsertHelper newInstance() {
+    return BulkInsertHelperHolder.SPARK_BULK_INSERT_HELPER;
+  }
+
+  @Override
+  public HoodieWriteMetadata<JavaRDD<WriteStatus>> bulkInsert(JavaRDD<HoodieRecord<T>> inputRecords,
+                                                              String instantTime,
+                                                              HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table,
+                                                              HoodieWriteConfig config,
+                                                              BaseCommitActionExecutor<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>, R> executor,
+                                                              boolean performDedupe,
+                                                              Option<BulkInsertPartitioner<T>> userDefinedBulkInsertPartitioner) {
+    HoodieWriteMetadata result = new HoodieWriteMetadata();
+
+    // De-dupe/merge if needed
+    JavaRDD<HoodieRecord<T>> dedupedRecords = inputRecords;
+
+    if (performDedupe) {
+      dedupedRecords = (JavaRDD<HoodieRecord<T>>) SparkWriteHelper.newInstance().combineOnCondition(config.shouldCombineBeforeInsert(), inputRecords,
+          config.getBulkInsertShuffleParallelism(), table);
+    }
+
+    final JavaRDD<HoodieRecord<T>> repartitionedRecords;
+    final int parallelism = config.getBulkInsertShuffleParallelism();
+    BulkInsertPartitioner partitioner = userDefinedBulkInsertPartitioner.isPresent()
+        ? userDefinedBulkInsertPartitioner.get()
+        : BulkInsertInternalPartitionerFactory.get(config.getBulkInsertSortMode());
+    repartitionedRecords = (JavaRDD<HoodieRecord<T>>) partitioner.repartitionRecords(dedupedRecords, parallelism);
+
+    // generate new file ID prefixes for each output partition
+    final List<String> fileIDPrefixes =
+        IntStream.range(0, parallelism).mapToObj(i -> FSUtils.createNewFileIdPfx()).collect(Collectors.toList());
+
+    table.getActiveTimeline().transitionRequestedToInflight(new HoodieInstant(HoodieInstant.State.REQUESTED,
+            table.getMetaClient().getCommitActionType(), instantTime), Option.empty(),
+        config.shouldAllowMultiWriteOnSameInstant());
+
+    JavaRDD<WriteStatus> writeStatusRDD = repartitionedRecords
+        .mapPartitionsWithIndex(new BulkInsertMapFunction<T>(instantTime,
+            partitioner.arePartitionRecordsSorted(), config, table, fileIDPrefixes), true)
+        .flatMap(List::iterator);
+
+    ((BaseSparkCommitActionExecutor) executor).updateIndexAndCommitIfNeeded(writeStatusRDD, result);
+    return result;
+  }
+}
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/BulkInsertPreppedCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertPreppedCommitActionExecutor.java
similarity index 67%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/commit/BulkInsertPreppedCommitActionExecutor.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertPreppedCommitActionExecutor.java
index f63d06e..e6b6809 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/BulkInsertPreppedCommitActionExecutor.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertPreppedCommitActionExecutor.java
@@ -18,6 +18,8 @@
 
 package org.apache.hudi.table.action.commit;
 
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.model.WriteOperationType;
@@ -29,27 +31,26 @@
 
 import org.apache.hudi.table.action.HoodieWriteMetadata;
 import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
 
-public class BulkInsertPreppedCommitActionExecutor<T extends HoodieRecordPayload<T>>
-    extends CommitActionExecutor<T> {
+public class SparkBulkInsertPreppedCommitActionExecutor<T extends HoodieRecordPayload<T>>
+    extends BaseSparkCommitActionExecutor<T> {
 
   private final JavaRDD<HoodieRecord<T>> preppedInputRecordRdd;
   private final Option<BulkInsertPartitioner<T>> userDefinedBulkInsertPartitioner;
 
-  public BulkInsertPreppedCommitActionExecutor(JavaSparkContext jsc,
-      HoodieWriteConfig config, HoodieTable table,
-      String instantTime, JavaRDD<HoodieRecord<T>> preppedInputRecordRdd,
-      Option<BulkInsertPartitioner<T>> userDefinedBulkInsertPartitioner) {
-    super(jsc, config, table, instantTime, WriteOperationType.BULK_INSERT);
+  public SparkBulkInsertPreppedCommitActionExecutor(HoodieSparkEngineContext context,
+                                                    HoodieWriteConfig config, HoodieTable table,
+                                                    String instantTime, JavaRDD<HoodieRecord<T>> preppedInputRecordRdd,
+                                                    Option<BulkInsertPartitioner<T>> userDefinedBulkInsertPartitioner) {
+    super(context, config, table, instantTime, WriteOperationType.BULK_INSERT);
     this.preppedInputRecordRdd = preppedInputRecordRdd;
     this.userDefinedBulkInsertPartitioner = userDefinedBulkInsertPartitioner;
   }
 
   @Override
-  public HoodieWriteMetadata execute() {
+  public HoodieWriteMetadata<JavaRDD<WriteStatus>> execute() {
     try {
-      return BulkInsertHelper.bulkInsert(preppedInputRecordRdd, instantTime, (HoodieTable<T>) table, config,
+      return SparkBulkInsertHelper.newInstance().bulkInsert(preppedInputRecordRdd, instantTime, table, config,
           this, false, userDefinedBulkInsertPartitioner);
     } catch (Throwable e) {
       if (e instanceof HoodieInsertException) {
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/DeleteCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteCommitActionExecutor.java
similarity index 63%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/commit/DeleteCommitActionExecutor.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteCommitActionExecutor.java
index 8fa1cb7..997c7bf 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/DeleteCommitActionExecutor.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteCommitActionExecutor.java
@@ -18,6 +18,8 @@
 
 package org.apache.hudi.table.action.commit;
 
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.model.HoodieKey;
 import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.model.WriteOperationType;
@@ -26,21 +28,21 @@
 
 import org.apache.hudi.table.action.HoodieWriteMetadata;
 import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
 
-public class DeleteCommitActionExecutor<T extends HoodieRecordPayload<T>>
-    extends CommitActionExecutor<T> {
+public class SparkDeleteCommitActionExecutor<T extends HoodieRecordPayload<T>>
+    extends BaseSparkCommitActionExecutor<T> {
 
   private final JavaRDD<HoodieKey> keys;
 
-  public DeleteCommitActionExecutor(JavaSparkContext jsc,
-      HoodieWriteConfig config, HoodieTable table,
-      String instantTime, JavaRDD<HoodieKey> keys) {
-    super(jsc, config, table, instantTime, WriteOperationType.DELETE);
+  public SparkDeleteCommitActionExecutor(HoodieSparkEngineContext context,
+                                         HoodieWriteConfig config, HoodieTable table,
+                                         String instantTime, JavaRDD<HoodieKey> keys) {
+    super(context, config, table, instantTime, WriteOperationType.DELETE);
     this.keys = keys;
   }
 
-  public HoodieWriteMetadata execute() {
-    return DeleteHelper.execute(instantTime, keys, jsc, config, (HoodieTable<T>)table, this);
+  @Override
+  public HoodieWriteMetadata<JavaRDD<WriteStatus>> execute() {
+    return SparkDeleteHelper.newInstance().execute(instantTime, keys, context, config, table, this);
   }
 }
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/DeleteHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteHelper.java
similarity index 63%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/commit/DeleteHelper.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteHelper.java
index b16bf63..01f9964 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/DeleteHelper.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteHelper.java
@@ -18,38 +18,49 @@
 
 package org.apache.hudi.table.action.commit;
 
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.model.EmptyHoodieRecordPayload;
 import org.apache.hudi.common.model.HoodieKey;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.util.collection.Pair;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.exception.HoodieUpsertException;
 import org.apache.hudi.table.HoodieTable;
 import org.apache.hudi.table.WorkloadProfile;
-
+import org.apache.hudi.table.WorkloadStat;
 import org.apache.hudi.table.action.HoodieWriteMetadata;
+
 import org.apache.spark.api.java.JavaRDD;
 import org.apache.spark.api.java.JavaSparkContext;
 
 import java.time.Duration;
 import java.time.Instant;
+import java.util.HashMap;
 
 /**
- * Helper class to perform delete keys on hoodie table.
+ * A spark implementation of {@link AbstractDeleteHelper}.
+ *
  * @param <T>
  */
-public class DeleteHelper<T extends HoodieRecordPayload<T>> {
+@SuppressWarnings("checkstyle:LineLength")
+public class SparkDeleteHelper<T extends HoodieRecordPayload,R> extends
+    AbstractDeleteHelper<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>, R> {
+  private SparkDeleteHelper() {
+  }
 
-  /**
-   * Deduplicate Hoodie records, using the given deduplication function.
-   *
-   * @param keys RDD of HoodieKey to deduplicate
-   * @param table target Hoodie table for deduplicating
-   * @param parallelism parallelism or partitions to be used while reducing/deduplicating
-   * @return RDD of HoodieKey already be deduplicated
-   */
-  private static  <T extends HoodieRecordPayload<T>> JavaRDD<HoodieKey> deduplicateKeys(JavaRDD<HoodieKey> keys,
-      HoodieTable<T> table, int parallelism) {
+  private static class DeleteHelperHolder {
+    private static final SparkDeleteHelper SPARK_DELETE_HELPER = new SparkDeleteHelper();
+  }
+
+  public static SparkDeleteHelper newInstance() {
+    return DeleteHelperHolder.SPARK_DELETE_HELPER;
+  }
+
+  @Override
+  public JavaRDD<HoodieKey> deduplicateKeys(JavaRDD<HoodieKey> keys, HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table, int parallelism) {
     boolean isIndexingGlobal = table.getIndex().isGlobal();
     if (isIndexingGlobal) {
       return keys.keyBy(HoodieKey::getRecordKey)
@@ -60,10 +71,15 @@
     }
   }
 
-  public static <T extends HoodieRecordPayload<T>> HoodieWriteMetadata execute(String instantTime,
-                                                                               JavaRDD<HoodieKey> keys, JavaSparkContext jsc,
-                                                                               HoodieWriteConfig config, HoodieTable<T> table,
-                                                                               CommitActionExecutor<T> deleteExecutor) {
+  @Override
+  public HoodieWriteMetadata<JavaRDD<WriteStatus>> execute(String instantTime,
+                                                           JavaRDD<HoodieKey> keys,
+                                                           HoodieEngineContext context,
+                                                           HoodieWriteConfig config,
+                                                           HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table,
+                                                           BaseCommitActionExecutor<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>, R> deleteExecutor) {
+    JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context);
+
     try {
       HoodieWriteMetadata result = null;
       JavaRDD<HoodieKey> dedupedKeys = keys;
@@ -80,7 +96,7 @@
       Instant beginTag = Instant.now();
       // perform index loop up to get existing location of records
       JavaRDD<HoodieRecord<T>> taggedRecords =
-          ((HoodieTable<T>)table).getIndex().tagLocation(dedupedRecords, jsc, (HoodieTable<T>)table);
+          table.getIndex().tagLocation(dedupedRecords, context, table);
       Duration tagLocationDuration = Duration.between(beginTag, Instant.now());
 
       // filter out non existent keys/records
@@ -90,7 +106,7 @@
         result.setIndexLookupDuration(tagLocationDuration);
       } else {
         // if entire set of keys are non existent
-        deleteExecutor.saveWorkloadProfileMetadataToInflight(new WorkloadProfile(jsc.emptyRDD()), instantTime);
+        deleteExecutor.saveWorkloadProfileMetadataToInflight(new WorkloadProfile(Pair.of(new HashMap<>(), new WorkloadStat())), instantTime);
         result = new HoodieWriteMetadata();
         result.setWriteStatuses(jsc.emptyRDD());
         deleteExecutor.commitOnAutoCommit(result);
@@ -103,4 +119,5 @@
       throw new HoodieUpsertException("Failed to delete for commit time " + instantTime, e);
     }
   }
+
 }
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/InsertCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertCommitActionExecutor.java
similarity index 66%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/commit/InsertCommitActionExecutor.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertCommitActionExecutor.java
index d8944e3..25891e0 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/InsertCommitActionExecutor.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertCommitActionExecutor.java
@@ -18,6 +18,8 @@
 
 package org.apache.hudi.table.action.commit;
 
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.model.WriteOperationType;
@@ -26,23 +28,22 @@
 
 import org.apache.hudi.table.action.HoodieWriteMetadata;
 import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
 
-public class InsertCommitActionExecutor<T extends HoodieRecordPayload<T>>
-    extends CommitActionExecutor<T> {
+public class SparkInsertCommitActionExecutor<T extends HoodieRecordPayload<T>>
+    extends BaseSparkCommitActionExecutor<T> {
 
   private final JavaRDD<HoodieRecord<T>> inputRecordsRDD;
 
-  public InsertCommitActionExecutor(JavaSparkContext jsc,
-      HoodieWriteConfig config, HoodieTable table,
-      String instantTime, JavaRDD<HoodieRecord<T>> inputRecordsRDD) {
-    super(jsc, config, table, instantTime, WriteOperationType.INSERT);
+  public SparkInsertCommitActionExecutor(HoodieSparkEngineContext context,
+                                         HoodieWriteConfig config, HoodieTable table,
+                                         String instantTime, JavaRDD<HoodieRecord<T>> inputRecordsRDD) {
+    super(context, config, table, instantTime, WriteOperationType.INSERT);
     this.inputRecordsRDD = inputRecordsRDD;
   }
 
   @Override
-  public HoodieWriteMetadata execute() {
-    return WriteHelper.write(instantTime, inputRecordsRDD, jsc, (HoodieTable<T>) table,
+  public HoodieWriteMetadata<JavaRDD<WriteStatus>> execute() {
+    return SparkWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table,
         config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(), this, false);
   }
 }
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/InsertOverwriteCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitActionExecutor.java
similarity index 75%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/commit/InsertOverwriteCommitActionExecutor.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitActionExecutor.java
index 054e8cd..627e75e 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/InsertOverwriteCommitActionExecutor.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitActionExecutor.java
@@ -19,6 +19,7 @@
 package org.apache.hudi.table.action.commit;
 
 import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieEngineContext;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.model.WriteOperationType;
@@ -31,36 +32,35 @@
 import org.apache.log4j.Logger;
 import org.apache.spark.Partitioner;
 import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
 import scala.Tuple2;
 
 import java.util.List;
 import java.util.Map;
 import java.util.stream.Collectors;
 
-public class InsertOverwriteCommitActionExecutor<T extends HoodieRecordPayload<T>>
-    extends CommitActionExecutor<T> {
+public class SparkInsertOverwriteCommitActionExecutor<T extends HoodieRecordPayload<T>>
+    extends BaseSparkCommitActionExecutor<T> {
 
-  private static final Logger LOG = LogManager.getLogger(InsertOverwriteCommitActionExecutor.class);
+  private static final Logger LOG = LogManager.getLogger(SparkInsertOverwriteCommitActionExecutor.class);
 
   private final JavaRDD<HoodieRecord<T>> inputRecordsRDD;
 
-  public InsertOverwriteCommitActionExecutor(JavaSparkContext jsc,
-                                             HoodieWriteConfig config, HoodieTable table,
-                                             String instantTime, JavaRDD<HoodieRecord<T>> inputRecordsRDD) {
-    super(jsc, config, table, instantTime, WriteOperationType.INSERT_OVERWRITE);
+  public SparkInsertOverwriteCommitActionExecutor(HoodieEngineContext context,
+                                                  HoodieWriteConfig config, HoodieTable table,
+                                                  String instantTime, JavaRDD<HoodieRecord<T>> inputRecordsRDD) {
+    super(context, config, table, instantTime, WriteOperationType.INSERT_OVERWRITE);
     this.inputRecordsRDD = inputRecordsRDD;
   }
 
   @Override
   public HoodieWriteMetadata execute() {
-    return WriteHelper.write(instantTime, inputRecordsRDD, jsc, (HoodieTable<T>) table,
+    return SparkWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table,
         config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(), this, false);
   }
 
   @Override
   protected Partitioner getPartitioner(WorkloadProfile profile) {
-    return new InsertOverwritePartitioner<>(profile, jsc, table, config);
+    return new SparkInsertOverwritePartitioner(profile, context, table, config);
   }
 
   @Override
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/InsertOverwritePartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwritePartitioner.java
similarity index 74%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/commit/InsertOverwritePartitioner.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwritePartitioner.java
index 80db612..6f8be79 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/InsertOverwritePartitioner.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwritePartitioner.java
@@ -18,13 +18,12 @@
 
 package org.apache.hudi.table.action.commit;
 
-import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.client.common.HoodieEngineContext;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.table.HoodieTable;
 import org.apache.hudi.table.WorkloadProfile;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
-import org.apache.spark.api.java.JavaSparkContext;
 
 import java.util.Collections;
 import java.util.List;
@@ -32,13 +31,13 @@
 /**
  * Packs incoming records to be inserted into buckets (1 bucket = 1 RDD partition).
  */
-public class InsertOverwritePartitioner<T extends HoodieRecordPayload<T>> extends UpsertPartitioner {
+public class SparkInsertOverwritePartitioner extends UpsertPartitioner {
 
-  private static final Logger LOG = LogManager.getLogger(InsertOverwritePartitioner.class);
+  private static final Logger LOG = LogManager.getLogger(SparkInsertOverwritePartitioner.class);
 
-  public InsertOverwritePartitioner(WorkloadProfile profile, JavaSparkContext jsc, HoodieTable<T> table,
-                                    HoodieWriteConfig config) {
-    super(profile, jsc, table, config);
+  public SparkInsertOverwritePartitioner(WorkloadProfile profile, HoodieEngineContext context, HoodieTable table,
+                                         HoodieWriteConfig config) {
+    super(profile, context, table, config);
   }
 
   /**
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/InsertPreppedCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertPreppedCommitActionExecutor.java
similarity index 66%
copy from hudi-client/src/main/java/org/apache/hudi/table/action/commit/InsertPreppedCommitActionExecutor.java
copy to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertPreppedCommitActionExecutor.java
index b7d64b1..400147b 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/InsertPreppedCommitActionExecutor.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertPreppedCommitActionExecutor.java
@@ -18,6 +18,8 @@
 
 package org.apache.hudi.table.action.commit;
 
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.model.WriteOperationType;
@@ -26,21 +28,21 @@
 
 import org.apache.hudi.table.action.HoodieWriteMetadata;
 import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
 
-public class InsertPreppedCommitActionExecutor<T extends HoodieRecordPayload<T>>
-    extends CommitActionExecutor<T> {
+public class SparkInsertPreppedCommitActionExecutor<T extends HoodieRecordPayload<T>>
+    extends BaseSparkCommitActionExecutor<T> {
 
   private final JavaRDD<HoodieRecord<T>> preppedRecords;
 
-  public InsertPreppedCommitActionExecutor(JavaSparkContext jsc,
-      HoodieWriteConfig config, HoodieTable table,
-      String instantTime, JavaRDD<HoodieRecord<T>> preppedRecords) {
-    super(jsc, config, table, instantTime, WriteOperationType.INSERT_PREPPED);
+  public SparkInsertPreppedCommitActionExecutor(HoodieSparkEngineContext context,
+                                                HoodieWriteConfig config, HoodieTable table,
+                                                String instantTime, JavaRDD<HoodieRecord<T>> preppedRecords) {
+    super(context, config, table, instantTime, WriteOperationType.INSERT_PREPPED);
     this.preppedRecords = preppedRecords;
   }
 
-  public HoodieWriteMetadata execute() {
+  @Override
+  public HoodieWriteMetadata<JavaRDD<WriteStatus>> execute() {
     return super.execute(preppedRecords);
   }
 }
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkMergeHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkMergeHelper.java
new file mode 100644
index 0000000..697b5ac
--- /dev/null
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkMergeHelper.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.hudi.table.action.commit;
+
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.model.HoodieBaseFile;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.execution.SparkBoundedInMemoryExecutor;
+import org.apache.hudi.io.HoodieMergeHandle;
+import org.apache.hudi.io.storage.HoodieFileReader;
+import org.apache.hudi.io.storage.HoodieFileReaderFactory;
+import org.apache.hudi.table.HoodieTable;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericDatumReader;
+import org.apache.avro.generic.GenericDatumWriter;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.io.BinaryDecoder;
+import org.apache.avro.io.BinaryEncoder;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.spark.api.java.JavaRDD;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+public class SparkMergeHelper<T extends HoodieRecordPayload> extends AbstractMergeHelper<T, JavaRDD<HoodieRecord<T>>,
+    JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
+
+  private static class MergeHelperHolder {
+    private static final SparkMergeHelper SPARK_MERGE_HELPER = new SparkMergeHelper();
+  }
+
+  public static SparkMergeHelper newInstance() {
+    return SparkMergeHelper.MergeHelperHolder.SPARK_MERGE_HELPER;
+  }
+
+  @Override
+  public void runMerge(HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table,
+                       HoodieMergeHandle<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> upsertHandle) throws IOException {
+    final boolean externalSchemaTransformation = table.getConfig().shouldUseExternalSchemaTransformation();
+    Configuration cfgForHoodieFile = new Configuration(table.getHadoopConf());
+    HoodieMergeHandle<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> mergeHandle = upsertHandle;
+    HoodieBaseFile baseFile = mergeHandle.baseFileForMerge();
+
+    final GenericDatumWriter<GenericRecord> gWriter;
+    final GenericDatumReader<GenericRecord> gReader;
+    Schema readSchema;
+    if (externalSchemaTransformation || baseFile.getBootstrapBaseFile().isPresent()) {
+      readSchema = HoodieFileReaderFactory.getFileReader(table.getHadoopConf(), mergeHandle.getOldFilePath()).getSchema();
+      gWriter = new GenericDatumWriter<>(readSchema);
+      gReader = new GenericDatumReader<>(readSchema, mergeHandle.getWriterSchemaWithMetafields());
+    } else {
+      gReader = null;
+      gWriter = null;
+      readSchema = mergeHandle.getWriterSchemaWithMetafields();
+    }
+
+    BoundedInMemoryExecutor<GenericRecord, GenericRecord, Void> wrapper = null;
+    HoodieFileReader<GenericRecord> reader = HoodieFileReaderFactory.<T, GenericRecord>getFileReader(cfgForHoodieFile, mergeHandle.getOldFilePath());
+    try {
+      final Iterator<GenericRecord> readerIterator;
+      if (baseFile.getBootstrapBaseFile().isPresent()) {
+        readerIterator = getMergingIterator(table, mergeHandle, baseFile, reader, readSchema, externalSchemaTransformation);
+      } else {
+        readerIterator = reader.getRecordIterator(readSchema);
+      }
+
+      ThreadLocal<BinaryEncoder> encoderCache = new ThreadLocal<>();
+      ThreadLocal<BinaryDecoder> decoderCache = new ThreadLocal<>();
+      wrapper = new SparkBoundedInMemoryExecutor(table.getConfig(), readerIterator,
+          new UpdateHandler(mergeHandle), record -> {
+        if (!externalSchemaTransformation) {
+          return record;
+        }
+        return transformRecordBasedOnNewSchema(gReader, gWriter, encoderCache, decoderCache, (GenericRecord) record);
+      });
+      wrapper.execute();
+    } catch (Exception e) {
+      throw new HoodieException(e);
+    } finally {
+      if (reader != null) {
+        reader.close();
+      }
+      mergeHandle.close();
+      if (null != wrapper) {
+        wrapper.shutdownNow();
+      }
+    }
+  }
+}
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/UpsertCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkUpsertCommitActionExecutor.java
similarity index 66%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/commit/UpsertCommitActionExecutor.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkUpsertCommitActionExecutor.java
index 0c4d08e..fe90212 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/UpsertCommitActionExecutor.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkUpsertCommitActionExecutor.java
@@ -18,6 +18,8 @@
 
 package org.apache.hudi.table.action.commit;
 
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.model.WriteOperationType;
@@ -26,23 +28,22 @@
 
 import org.apache.hudi.table.action.HoodieWriteMetadata;
 import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
 
-public class UpsertCommitActionExecutor<T extends HoodieRecordPayload<T>>
-    extends CommitActionExecutor<T> {
+public class SparkUpsertCommitActionExecutor<T extends HoodieRecordPayload<T>>
+    extends BaseSparkCommitActionExecutor<T> {
 
   private JavaRDD<HoodieRecord<T>> inputRecordsRDD;
 
-  public UpsertCommitActionExecutor(JavaSparkContext jsc,
-      HoodieWriteConfig config, HoodieTable table,
-      String instantTime, JavaRDD<HoodieRecord<T>> inputRecordsRDD) {
-    super(jsc, config, table, instantTime, WriteOperationType.UPSERT);
+  public SparkUpsertCommitActionExecutor(HoodieSparkEngineContext context,
+                                         HoodieWriteConfig config, HoodieTable table,
+                                         String instantTime, JavaRDD<HoodieRecord<T>> inputRecordsRDD) {
+    super(context, config, table, instantTime, WriteOperationType.UPSERT);
     this.inputRecordsRDD = inputRecordsRDD;
   }
 
   @Override
-  public HoodieWriteMetadata execute() {
-    return WriteHelper.write(instantTime, inputRecordsRDD, jsc, (HoodieTable<T>)table,
+  public HoodieWriteMetadata<JavaRDD<WriteStatus>> execute() {
+    return SparkWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table,
         config.shouldCombineBeforeUpsert(), config.getUpsertShuffleParallelism(), this, true);
   }
 }
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/InsertPreppedCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkUpsertPreppedCommitActionExecutor.java
similarity index 66%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/commit/InsertPreppedCommitActionExecutor.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkUpsertPreppedCommitActionExecutor.java
index b7d64b1..e36073f 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/InsertPreppedCommitActionExecutor.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkUpsertPreppedCommitActionExecutor.java
@@ -18,6 +18,8 @@
 
 package org.apache.hudi.table.action.commit;
 
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.model.WriteOperationType;
@@ -26,21 +28,21 @@
 
 import org.apache.hudi.table.action.HoodieWriteMetadata;
 import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
 
-public class InsertPreppedCommitActionExecutor<T extends HoodieRecordPayload<T>>
-    extends CommitActionExecutor<T> {
+public class SparkUpsertPreppedCommitActionExecutor<T extends HoodieRecordPayload<T>>
+    extends BaseSparkCommitActionExecutor<T> {
 
   private final JavaRDD<HoodieRecord<T>> preppedRecords;
 
-  public InsertPreppedCommitActionExecutor(JavaSparkContext jsc,
-      HoodieWriteConfig config, HoodieTable table,
-      String instantTime, JavaRDD<HoodieRecord<T>> preppedRecords) {
-    super(jsc, config, table, instantTime, WriteOperationType.INSERT_PREPPED);
+  public SparkUpsertPreppedCommitActionExecutor(HoodieSparkEngineContext context,
+                                                HoodieWriteConfig config, HoodieTable table,
+                                                String instantTime, JavaRDD<HoodieRecord<T>> preppedRecords) {
+    super(context, config, table, instantTime, WriteOperationType.UPSERT_PREPPED);
     this.preppedRecords = preppedRecords;
   }
 
-  public HoodieWriteMetadata execute() {
+  @Override
+  public HoodieWriteMetadata<JavaRDD<WriteStatus>> execute() {
     return super.execute(preppedRecords);
   }
 }
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkWriteHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkWriteHelper.java
new file mode 100644
index 0000000..5f1a1ef
--- /dev/null
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkWriteHelper.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.hudi.table.action.commit;
+
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.index.HoodieIndex;
+
+import org.apache.spark.api.java.JavaRDD;
+
+import scala.Tuple2;
+
+/**
+ * A spark implementation of {@link AbstractWriteHelper}.
+ *
+ * @param <T>
+ */
+public class SparkWriteHelper<T extends HoodieRecordPayload,R> extends AbstractWriteHelper<T, JavaRDD<HoodieRecord<T>>,
+    JavaRDD<HoodieKey>, JavaRDD<WriteStatus>, R> {
+  private SparkWriteHelper() {
+  }
+
+  private static class WriteHelperHolder {
+    private static final SparkWriteHelper SPARK_WRITE_HELPER = new SparkWriteHelper();
+  }
+
+  public static SparkWriteHelper newInstance() {
+    return WriteHelperHolder.SPARK_WRITE_HELPER;
+  }
+
+  @Override
+  public JavaRDD<HoodieRecord<T>> deduplicateRecords(JavaRDD<HoodieRecord<T>> records,
+                                                     HoodieIndex<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> index,
+                                                     int parallelism) {
+    boolean isIndexingGlobal = index.isGlobal();
+    return records.mapToPair(record -> {
+      HoodieKey hoodieKey = record.getKey();
+      // If index used is global, then records are expected to differ in their partitionPath
+      Object key = isIndexingGlobal ? hoodieKey.getRecordKey() : hoodieKey;
+      return new Tuple2<>(key, record);
+    }).reduceByKey((rec1, rec2) -> {
+      @SuppressWarnings("unchecked")
+      T reducedData = (T) rec1.getData().preCombine(rec2.getData());
+      // we cannot allow the user to change the key or partitionPath, since that will affect
+      // everything
+      // so pick it from one of the records.
+      return new HoodieRecord<T>(rec1.getKey(), reducedData);
+    }, parallelism).map(Tuple2::_2);
+  }
+
+}
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java
similarity index 95%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java
index 86fa1bf..b28c89a 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java
@@ -18,6 +18,8 @@
 
 package org.apache.hudi.table.action.commit;
 
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.HoodieBaseFile;
 import org.apache.hudi.common.model.HoodieCommitMetadata;
@@ -84,11 +86,11 @@
    */
   private HashMap<Integer, BucketInfo> bucketInfoMap;
 
-  protected final HoodieTable<T> table;
+  protected final HoodieTable table;
 
   protected final HoodieWriteConfig config;
 
-  public UpsertPartitioner(WorkloadProfile profile, JavaSparkContext jsc, HoodieTable<T> table,
+  public UpsertPartitioner(WorkloadProfile profile, HoodieEngineContext context, HoodieTable table,
       HoodieWriteConfig config) {
     updateLocationToBucket = new HashMap<>();
     partitionPathToInsertBucketInfos = new HashMap<>();
@@ -97,7 +99,7 @@
     this.table = table;
     this.config = config;
     assignUpdates(profile);
-    assignInserts(profile, jsc);
+    assignInserts(profile, context);
 
     LOG.info("Total Buckets :" + totalBuckets + ", buckets info => " + bucketInfoMap + ", \n"
         + "Partition to insert buckets => " + partitionPathToInsertBucketInfos + ", \n"
@@ -127,7 +129,7 @@
     return bucket;
   }
 
-  private void assignInserts(WorkloadProfile profile, JavaSparkContext jsc) {
+  private void assignInserts(WorkloadProfile profile, HoodieEngineContext context) {
     // for new inserts, compute buckets depending on how many records we have for each partition
     Set<String> partitionPaths = profile.getPartitionPaths();
     long averageRecordSize =
@@ -136,7 +138,7 @@
     LOG.info("AvgRecordSize => " + averageRecordSize);
 
     Map<String, List<SmallFile>> partitionSmallFilesMap =
-        getSmallFilesForPartitions(new ArrayList<String>(partitionPaths), jsc);
+        getSmallFilesForPartitions(new ArrayList<String>(partitionPaths), context);
 
     for (String partitionPath : partitionPaths) {
       WorkloadStat pStat = profile.getWorkloadStat(partitionPath);
@@ -209,11 +211,11 @@
     }
   }
 
-  private Map<String, List<SmallFile>> getSmallFilesForPartitions(List<String> partitionPaths, JavaSparkContext jsc) {
-
+  private Map<String, List<SmallFile>> getSmallFilesForPartitions(List<String> partitionPaths, HoodieEngineContext context) {
+    JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context);
     Map<String, List<SmallFile>> partitionSmallFilesMap = new HashMap<>();
     if (partitionPaths != null && partitionPaths.size() > 0) {
-      jsc.setJobGroup(this.getClass().getSimpleName(), "Getting small files from partitions");
+      context.setJobStatus(this.getClass().getSimpleName(), "Getting small files from partitions");
       JavaRDD<String> partitionPathRdds = jsc.parallelize(partitionPaths, partitionPaths.size());
       partitionSmallFilesMap = partitionPathRdds.mapToPair((PairFunction<String, String, List<SmallFile>>)
           partitionPath -> new Tuple2<>(partitionPath, getSmallFiles(partitionPath))).collectAsMap();
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/compact/HoodieMergeOnReadTableCompactor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/HoodieSparkMergeOnReadTableCompactor.java
similarity index 75%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/compact/HoodieMergeOnReadTableCompactor.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/HoodieSparkMergeOnReadTableCompactor.java
index c4343f8..505eabb 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/compact/HoodieMergeOnReadTableCompactor.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/HoodieSparkMergeOnReadTableCompactor.java
@@ -21,13 +21,18 @@
 import org.apache.hudi.avro.HoodieAvroUtils;
 import org.apache.hudi.avro.model.HoodieCompactionOperation;
 import org.apache.hudi.avro.model.HoodieCompactionPlan;
+import org.apache.hudi.client.SparkTaskContextSupplier;
 import org.apache.hudi.client.WriteStatus;
-import org.apache.hudi.client.utils.SparkConfigUtils;
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.CompactionOperation;
 import org.apache.hudi.common.model.HoodieBaseFile;
 import org.apache.hudi.common.model.HoodieFileGroupId;
+import org.apache.hudi.common.model.HoodieKey;
 import org.apache.hudi.common.model.HoodieLogFile;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.model.HoodieTableType;
 import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
@@ -40,7 +45,8 @@
 import org.apache.hudi.common.util.ValidationUtils;
 import org.apache.hudi.common.util.collection.Pair;
 import org.apache.hudi.config.HoodieWriteConfig;
-import org.apache.hudi.table.HoodieCopyOnWriteTable;
+import org.apache.hudi.io.IOUtils;
+import org.apache.hudi.table.HoodieSparkCopyOnWriteTable;
 import org.apache.hudi.table.HoodieTable;
 import org.apache.hudi.table.action.compact.strategy.CompactionStrategy;
 
@@ -51,7 +57,6 @@
 import org.apache.log4j.Logger;
 import org.apache.spark.api.java.JavaRDD;
 import org.apache.spark.api.java.JavaSparkContext;
-import org.apache.spark.api.java.function.FlatMapFunction;
 import org.apache.spark.util.AccumulatorV2;
 import org.apache.spark.util.LongAccumulator;
 
@@ -72,34 +77,36 @@
  * a normal commit
  *
  */
-public class HoodieMergeOnReadTableCompactor implements HoodieCompactor {
+@SuppressWarnings("checkstyle:LineLength")
+public class HoodieSparkMergeOnReadTableCompactor<T extends HoodieRecordPayload> implements HoodieCompactor<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
 
-  private static final Logger LOG = LogManager.getLogger(HoodieMergeOnReadTableCompactor.class);
+  private static final Logger LOG = LogManager.getLogger(HoodieSparkMergeOnReadTableCompactor.class);
   // Accumulator to keep track of total log files for a table
   private AccumulatorV2<Long, Long> totalLogFiles;
   // Accumulator to keep track of total log file slices for a table
   private AccumulatorV2<Long, Long> totalFileSlices;
 
   @Override
-  public JavaRDD<WriteStatus> compact(JavaSparkContext jsc, HoodieCompactionPlan compactionPlan,
-      HoodieTable hoodieTable, HoodieWriteConfig config, String compactionInstantTime) throws IOException {
+  public JavaRDD<WriteStatus> compact(HoodieEngineContext context, HoodieCompactionPlan compactionPlan,
+                                      HoodieTable hoodieTable, HoodieWriteConfig config, String compactionInstantTime) throws IOException {
+    JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context);
     if (compactionPlan == null || (compactionPlan.getOperations() == null)
         || (compactionPlan.getOperations().isEmpty())) {
       return jsc.emptyRDD();
     }
     HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
     // Compacting is very similar to applying updates to existing file
-    HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc.hadoopConfiguration(), metaClient);
+    HoodieSparkCopyOnWriteTable table = new HoodieSparkCopyOnWriteTable(config, context, metaClient);
     List<CompactionOperation> operations = compactionPlan.getOperations().stream()
         .map(CompactionOperation::convertFromAvroRecordInstance).collect(toList());
     LOG.info("Compactor compacting " + operations + " files");
 
-    jsc.setJobGroup(this.getClass().getSimpleName(), "Compacting file slices");
+    context.setJobStatus(this.getClass().getSimpleName(), "Compacting file slices");
     return jsc.parallelize(operations, operations.size())
         .map(s -> compact(table, metaClient, config, s, compactionInstantTime)).flatMap(List::iterator);
   }
 
-  private List<WriteStatus> compact(HoodieCopyOnWriteTable hoodieCopyOnWriteTable, HoodieTableMetaClient metaClient,
+  private List<WriteStatus> compact(HoodieSparkCopyOnWriteTable hoodieCopyOnWriteTable, HoodieTableMetaClient metaClient,
       HoodieWriteConfig config, CompactionOperation operation, String instantTime) throws IOException {
     FileSystem fs = metaClient.getFs();
 
@@ -116,12 +123,12 @@
         .getActiveTimeline().getTimelineOfActions(CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION,
             HoodieTimeline.ROLLBACK_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION))
         .filterCompletedInstants().lastInstant().get().getTimestamp();
-    LOG.info("MaxMemoryPerCompaction => " + SparkConfigUtils.getMaxMemoryPerCompaction(config.getProps()));
+    long maxMemoryPerCompaction = IOUtils.getMaxMemoryPerCompaction(new SparkTaskContextSupplier(), config.getProps());
+    LOG.info("MaxMemoryPerCompaction => " + maxMemoryPerCompaction);
 
     List<String> logFiles = operation.getDeltaFileNames().stream().map(
         p -> new Path(FSUtils.getPartitionPath(metaClient.getBasePath(), operation.getPartitionPath()), p).toString())
         .collect(toList());
-    long maxMemoryPerCompaction = SparkConfigUtils.getMaxMemoryPerCompaction(config.getProps());
     HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, metaClient.getBasePath(), logFiles,
         readerSchema, maxInstantTime, maxMemoryPerCompaction, config.getCompactionLazyBlockReadEnabled(),
         config.getCompactionReverseLogReadEnabled(), config.getMaxDFSStreamBufferSize(),
@@ -162,10 +169,11 @@
   }
 
   @Override
-  public HoodieCompactionPlan generateCompactionPlan(JavaSparkContext jsc, HoodieTable hoodieTable,
-      HoodieWriteConfig config, String compactionCommitTime, Set<HoodieFileGroupId> fgIdsInPendingCompactions)
+  public HoodieCompactionPlan generateCompactionPlan(HoodieEngineContext context,
+                                                     HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable,
+                                                     HoodieWriteConfig config, String compactionCommitTime, Set<HoodieFileGroupId> fgIdsInPendingCompactions)
       throws IOException {
-
+    JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context);
     totalLogFiles = new LongAccumulator();
     totalFileSlices = new LongAccumulator();
     jsc.sc().register(totalLogFiles);
@@ -192,23 +200,27 @@
 
     SliceView fileSystemView = hoodieTable.getSliceView();
     LOG.info("Compaction looking for files to compact in " + partitionPaths + " partitions");
-    jsc.setJobGroup(this.getClass().getSimpleName(), "Looking for files to compact");
-    List<HoodieCompactionOperation> operations = jsc.parallelize(partitionPaths, partitionPaths.size())
-        .flatMap((FlatMapFunction<String, CompactionOperation>) partitionPath -> fileSystemView
-            .getLatestFileSlices(partitionPath)
-            .filter(slice -> !fgIdsInPendingCompactions.contains(slice.getFileGroupId())).map(s -> {
-              List<HoodieLogFile> logFiles =
-                  s.getLogFiles().sorted(HoodieLogFile.getLogFileComparator()).collect(Collectors.toList());
-              totalLogFiles.add((long) logFiles.size());
-              totalFileSlices.add(1L);
-              // Avro generated classes are not inheriting Serializable. Using CompactionOperation POJO
-              // for spark Map operations and collecting them finally in Avro generated classes for storing
-              // into meta files.
-              Option<HoodieBaseFile> dataFile = s.getBaseFile();
-              return new CompactionOperation(dataFile, partitionPath, logFiles,
-                  config.getCompactionStrategy().captureMetrics(config, dataFile, partitionPath, logFiles));
-            }).filter(c -> !c.getDeltaFileNames().isEmpty()).collect(toList()).iterator())
-        .collect().stream().map(CompactionUtils::buildHoodieCompactionOperation).collect(toList());
+    context.setJobStatus(this.getClass().getSimpleName(), "Looking for files to compact");
+
+    List<HoodieCompactionOperation> operations = context.flatMap(partitionPaths, partitionPath -> {
+      return fileSystemView
+          .getLatestFileSlices(partitionPath)
+          .filter(slice -> !fgIdsInPendingCompactions.contains(slice.getFileGroupId()))
+          .map(s -> {
+            List<HoodieLogFile> logFiles =
+                s.getLogFiles().sorted(HoodieLogFile.getLogFileComparator()).collect(Collectors.toList());
+            totalLogFiles.add((long) logFiles.size());
+            totalFileSlices.add(1L);
+            // Avro generated classes are not inheriting Serializable. Using CompactionOperation POJO
+            // for spark Map operations and collecting them finally in Avro generated classes for storing
+            // into meta files.
+            Option<HoodieBaseFile> dataFile = s.getBaseFile();
+            return new CompactionOperation(dataFile, partitionPath, logFiles,
+                config.getCompactionStrategy().captureMetrics(config, dataFile, partitionPath, logFiles));
+          })
+          .filter(c -> !c.getDeltaFileNames().isEmpty());
+    }, partitionPaths.size()).stream().map(CompactionUtils::buildHoodieCompactionOperation).collect(toList());
+
     LOG.info("Total of " + operations.size() + " compactions are retrieved");
     LOG.info("Total number of latest files slices " + totalFileSlices.value());
     LOG.info("Total number of log files " + totalLogFiles.value());
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/compact/CompactHelpers.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkCompactHelpers.java
similarity index 61%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/compact/CompactHelpers.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkCompactHelpers.java
index 97fdd0f..107f533 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/compact/CompactHelpers.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkCompactHelpers.java
@@ -21,26 +21,43 @@
 import org.apache.hudi.avro.model.HoodieCompactionPlan;
 import org.apache.hudi.client.WriteStatus;
 import org.apache.hudi.common.model.HoodieCommitMetadata;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.model.HoodieWriteStat;
-import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
-import org.apache.hudi.common.table.timeline.HoodieInstant;
 import org.apache.hudi.common.table.timeline.HoodieTimeline;
 import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
-import org.apache.hudi.common.util.Option;
-import org.apache.hudi.exception.HoodieCompactionException;
 import org.apache.hudi.table.HoodieTable;
+
 import org.apache.spark.api.java.JavaRDD;
 
 import java.io.IOException;
-import java.nio.charset.StandardCharsets;
 import java.util.List;
 
-public class CompactHelpers {
+/**
+ * A spark implementation of {@link AbstractCompactHelpers}.
+ *
+ * @param <T>
+ */
+public class SparkCompactHelpers<T extends HoodieRecordPayload> extends
+    AbstractCompactHelpers<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
 
-  public static HoodieCommitMetadata createCompactionMetadata(HoodieTable<?> table,
-                                                              String compactionInstantTime,
-                                                              JavaRDD<WriteStatus> writeStatuses,
-                                                              String schema) throws IOException {
+  private SparkCompactHelpers() {
+  }
+
+  private static class CompactHelperHolder {
+    private static final SparkCompactHelpers SPARK_COMPACT_HELPERS = new SparkCompactHelpers();
+  }
+
+  public static SparkCompactHelpers newInstance() {
+    return CompactHelperHolder.SPARK_COMPACT_HELPERS;
+  }
+
+  @Override
+  public HoodieCommitMetadata createCompactionMetadata(HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table,
+                                                       String compactionInstantTime,
+                                                       JavaRDD<WriteStatus> writeStatuses,
+                                                       String schema) throws IOException {
     byte[] planBytes = table.getActiveTimeline().readCompactionPlanAsBytes(
         HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime)).get();
     HoodieCompactionPlan compactionPlan = TimelineMetadataUtils.deserializeCompactionPlan(planBytes);
@@ -55,16 +72,4 @@
     }
     return metadata;
   }
-
-  public static void completeInflightCompaction(HoodieTable<?> table, String compactionCommitTime, HoodieCommitMetadata commitMetadata) {
-    HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
-    try {
-      activeTimeline.transitionCompactionInflightToComplete(
-          new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionCommitTime),
-          Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
-    } catch (IOException e) {
-      throw new HoodieCompactionException(
-          "Failed to commit " + table.getMetaClient().getBasePath() + " at time " + compactionCommitTime, e);
-    }
-  }
 }
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/compact/RunCompactionActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkRunCompactionActionExecutor.java
similarity index 70%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/compact/RunCompactionActionExecutor.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkRunCompactionActionExecutor.java
index 2f99fa1..ebc3de5 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/compact/RunCompactionActionExecutor.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkRunCompactionActionExecutor.java
@@ -20,8 +20,12 @@
 
 import org.apache.hudi.avro.model.HoodieCompactionPlan;
 import org.apache.hudi.client.WriteStatus;
-import org.apache.hudi.client.utils.SparkConfigUtils;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
+import org.apache.hudi.client.utils.SparkMemoryUtils;
 import org.apache.hudi.common.model.HoodieCommitMetadata;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.model.HoodieWriteStat;
 import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
 import org.apache.hudi.common.table.timeline.HoodieInstant;
@@ -33,27 +37,25 @@
 import org.apache.hudi.table.HoodieTable;
 import org.apache.hudi.table.action.BaseActionExecutor;
 import org.apache.hudi.table.action.HoodieWriteMetadata;
-import org.apache.log4j.LogManager;
-import org.apache.log4j.Logger;
+
 import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
 
 import java.io.IOException;
 import java.util.List;
 
-public class RunCompactionActionExecutor extends BaseActionExecutor<HoodieWriteMetadata> {
+@SuppressWarnings("checkstyle:LineLength")
+public class SparkRunCompactionActionExecutor<T extends HoodieRecordPayload> extends
+    BaseActionExecutor<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>, HoodieWriteMetadata<JavaRDD<WriteStatus>>> {
 
-  private static final Logger LOG = LogManager.getLogger(RunCompactionActionExecutor.class);
-
-  public RunCompactionActionExecutor(JavaSparkContext jsc,
-                                     HoodieWriteConfig config,
-                                     HoodieTable<?> table,
-                                     String instantTime) {
-    super(jsc, config, table, instantTime);
+  public SparkRunCompactionActionExecutor(HoodieSparkEngineContext context,
+                                          HoodieWriteConfig config,
+                                          HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table,
+                                          String instantTime) {
+    super(context, config, table, instantTime);
   }
 
   @Override
-  public HoodieWriteMetadata execute() {
+  public HoodieWriteMetadata<JavaRDD<WriteStatus>> execute() {
     HoodieInstant instant = HoodieTimeline.getCompactionRequestedInstant(instantTime);
     HoodieTimeline pendingCompactionTimeline = table.getActiveTimeline().filterPendingCompactionTimeline();
     if (!pendingCompactionTimeline.containsInstant(instant)) {
@@ -61,7 +63,7 @@
           "No Compaction request available at " + instantTime + " to run compaction");
     }
 
-    HoodieWriteMetadata compactionMetadata = new HoodieWriteMetadata();
+    HoodieWriteMetadata<JavaRDD<WriteStatus>> compactionMetadata = new HoodieWriteMetadata<>();
     try {
       HoodieActiveTimeline timeline = table.getActiveTimeline();
       HoodieCompactionPlan compactionPlan =
@@ -70,10 +72,10 @@
       timeline.transitionCompactionRequestedToInflight(instant);
       table.getMetaClient().reloadActiveTimeline();
 
-      HoodieMergeOnReadTableCompactor compactor = new HoodieMergeOnReadTableCompactor();
-      JavaRDD<WriteStatus> statuses = compactor.compact(jsc, compactionPlan, table, config, instantTime);
+      HoodieSparkMergeOnReadTableCompactor compactor = new HoodieSparkMergeOnReadTableCompactor();
+      JavaRDD<WriteStatus> statuses = compactor.compact(context, compactionPlan, table, config, instantTime);
 
-      statuses.persist(SparkConfigUtils.getWriteStatusStorageLevel(config.getProps()));
+      statuses.persist(SparkMemoryUtils.getWriteStatusStorageLevel(config.getProps()));
       List<HoodieWriteStat> updateStatusMap = statuses.map(WriteStatus::getStat).collect();
       HoodieCommitMetadata metadata = new HoodieCommitMetadata(true);
       for (HoodieWriteStat stat : updateStatusMap) {
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
new file mode 100644
index 0000000..c5f6c16
--- /dev/null
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/SparkScheduleCompactionActionExecutor.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.table.action.compact;
+
+import org.apache.hudi.avro.model.HoodieCompactionPlan;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.view.SyncableFileSystemView;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieCompactionException;
+import org.apache.hudi.table.HoodieTable;
+
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaRDD;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+@SuppressWarnings("checkstyle:LineLength")
+public class SparkScheduleCompactionActionExecutor<T extends HoodieRecordPayload> extends
+    BaseScheduleCompactionActionExecutor<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
+
+  private static final Logger LOG = LogManager.getLogger(SparkScheduleCompactionActionExecutor.class);
+
+  public SparkScheduleCompactionActionExecutor(HoodieEngineContext context,
+                                               HoodieWriteConfig config,
+                                               HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table,
+                                               String instantTime,
+                                               Option<Map<String, String>> extraMetadata) {
+    super(context, config, table, instantTime, extraMetadata);
+  }
+
+  @Override
+  protected HoodieCompactionPlan scheduleCompaction() {
+    LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
+    Option<HoodieInstant> lastCompaction = table.getActiveTimeline().getCommitTimeline()
+        .filterCompletedInstants().lastInstant();
+    String lastCompactionTs = "0";
+    if (lastCompaction.isPresent()) {
+      lastCompactionTs = lastCompaction.get().getTimestamp();
+    }
+
+    int deltaCommitsSinceLastCompaction = table.getActiveTimeline().getDeltaCommitTimeline()
+        .findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
+    if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction) {
+      LOG.info("Not scheduling compaction as only " + deltaCommitsSinceLastCompaction
+          + " delta commits was found since last compaction " + lastCompactionTs + ". Waiting for "
+          + config.getInlineCompactDeltaCommitMax());
+      return new HoodieCompactionPlan();
+    }
+
+    LOG.info("Generating compaction plan for merge on read table " + config.getBasePath());
+    HoodieSparkMergeOnReadTableCompactor compactor = new HoodieSparkMergeOnReadTableCompactor();
+    try {
+      return compactor.generateCompactionPlan(context, table, config, instantTime,
+          ((SyncableFileSystemView) table.getSliceView()).getPendingCompactionOperations()
+              .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId())
+              .collect(Collectors.toSet()));
+
+    } catch (IOException e) {
+      throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e);
+    }
+  }
+
+}
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/DeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/AbstractSparkDeltaCommitActionExecutor.java
similarity index 64%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/DeltaCommitActionExecutor.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/AbstractSparkDeltaCommitActionExecutor.java
index f616a00..64d4c9c 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/DeltaCommitActionExecutor.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/AbstractSparkDeltaCommitActionExecutor.java
@@ -20,45 +20,45 @@
 
 import java.util.Map;
 import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.model.WriteOperationType;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.exception.HoodieUpsertException;
-import org.apache.hudi.execution.LazyInsertIterable;
+import org.apache.hudi.execution.SparkLazyInsertIterable;
 import org.apache.hudi.io.AppendHandleFactory;
 import org.apache.hudi.io.HoodieAppendHandle;
 import org.apache.hudi.table.HoodieTable;
 import org.apache.hudi.table.WorkloadProfile;
-import org.apache.hudi.table.action.commit.CommitActionExecutor;
+import org.apache.hudi.table.action.commit.BaseSparkCommitActionExecutor;
 
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
 import org.apache.spark.Partitioner;
-import org.apache.spark.api.java.JavaSparkContext;
 
 import java.io.IOException;
 import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 
-public abstract class DeltaCommitActionExecutor<T extends HoodieRecordPayload<T>>
-    extends CommitActionExecutor<T> {
-  private static final Logger LOG = LogManager.getLogger(DeltaCommitActionExecutor.class);
+public abstract class AbstractSparkDeltaCommitActionExecutor<T extends HoodieRecordPayload<T>>
+    extends BaseSparkCommitActionExecutor<T> {
+  private static final Logger LOG = LogManager.getLogger(AbstractSparkDeltaCommitActionExecutor.class);
 
   // UpsertPartitioner for MergeOnRead table type
-  private UpsertDeltaCommitPartitioner mergeOnReadUpsertPartitioner;
+  private SparkUpsertDeltaCommitPartitioner mergeOnReadUpsertPartitioner;
 
-  public DeltaCommitActionExecutor(JavaSparkContext jsc, HoodieWriteConfig config, HoodieTable table,
-                                   String instantTime, WriteOperationType operationType) {
-    this(jsc, config, table, instantTime, operationType, Option.empty());
+  public AbstractSparkDeltaCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table,
+                                                String instantTime, WriteOperationType operationType) {
+    this(context, config, table, instantTime, operationType, Option.empty());
   }
 
-  public DeltaCommitActionExecutor(JavaSparkContext jsc, HoodieWriteConfig config, HoodieTable table,
-                                   String instantTime, WriteOperationType operationType,
-                                   Option<Map<String, String>> extraMetadata) {
-    super(jsc, config, table, instantTime, operationType, extraMetadata);
+  public AbstractSparkDeltaCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table,
+                                                String instantTime, WriteOperationType operationType,
+                                                Option<Map<String, String>> extraMetadata) {
+    super(context, config, table, instantTime, operationType, extraMetadata);
   }
 
   @Override
@@ -66,7 +66,7 @@
     if (profile == null) {
       throw new HoodieUpsertException("Need workload profile to construct the upsert partitioner.");
     }
-    mergeOnReadUpsertPartitioner = new UpsertDeltaCommitPartitioner(profile, jsc, table, config);
+    mergeOnReadUpsertPartitioner = new SparkUpsertDeltaCommitPartitioner(profile, (HoodieSparkEngineContext) context, table, config);
     return mergeOnReadUpsertPartitioner;
   }
 
@@ -79,8 +79,8 @@
       LOG.info("Small file corrections for updates for commit " + instantTime + " for file " + fileId);
       return super.handleUpdate(partitionPath, fileId, recordItr);
     } else {
-      HoodieAppendHandle<T> appendHandle = new HoodieAppendHandle<>(config, instantTime, (HoodieTable<T>)table,
-          partitionPath, fileId, recordItr, sparkTaskContextSupplier);
+      HoodieAppendHandle appendHandle = new HoodieAppendHandle<>(config, instantTime, table,
+          partitionPath, fileId, recordItr, taskContextSupplier);
       appendHandle.doAppend();
       appendHandle.close();
       return Collections.singletonList(Collections.singletonList(appendHandle.getWriteStatus())).iterator();
@@ -92,8 +92,8 @@
       throws Exception {
     // If canIndexLogFiles, write inserts to log files else write inserts to base files
     if (table.getIndex().canIndexLogFiles()) {
-      return new LazyInsertIterable<>(recordItr, true, config, instantTime, (HoodieTable<T>) table,
-          idPfx, sparkTaskContextSupplier, new AppendHandleFactory<>());
+      return new SparkLazyInsertIterable<>(recordItr, true, config, instantTime, table,
+          idPfx, taskContextSupplier, new AppendHandleFactory<>());
     } else {
       return super.handleInsert(idPfx, recordItr);
     }
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkBulkInsertDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkBulkInsertDeltaCommitActionExecutor.java
new file mode 100644
index 0000000..281304d
--- /dev/null
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkBulkInsertDeltaCommitActionExecutor.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.hudi.table.action.deltacommit;
+
+import java.util.Map;
+
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieInsertException;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.table.BulkInsertPartitioner;
+
+import org.apache.hudi.table.action.HoodieWriteMetadata;
+import org.apache.hudi.table.action.commit.SparkBulkInsertHelper;
+import org.apache.spark.api.java.JavaRDD;
+
+public class SparkBulkInsertDeltaCommitActionExecutor<T extends HoodieRecordPayload<T>>
+    extends AbstractSparkDeltaCommitActionExecutor<T> {
+
+  private final JavaRDD<HoodieRecord<T>> inputRecordsRDD;
+  private final Option<BulkInsertPartitioner<T>> bulkInsertPartitioner;
+
+  public SparkBulkInsertDeltaCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table,
+                                                  String instantTime, JavaRDD<HoodieRecord<T>> inputRecordsRDD,
+                                                  Option<BulkInsertPartitioner<T>> bulkInsertPartitioner)  {
+    this(context, config, table, instantTime, inputRecordsRDD, bulkInsertPartitioner, Option.empty());
+  }
+
+  public SparkBulkInsertDeltaCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table,
+                                                  String instantTime, JavaRDD<HoodieRecord<T>> inputRecordsRDD,
+                                                  Option<BulkInsertPartitioner<T>> bulkInsertPartitioner,
+                                                  Option<Map<String, String>> extraMetadata) {
+    super(context, config, table, instantTime, WriteOperationType.BULK_INSERT, extraMetadata);
+    this.inputRecordsRDD = inputRecordsRDD;
+    this.bulkInsertPartitioner = bulkInsertPartitioner;
+  }
+
+  @Override
+  public HoodieWriteMetadata<JavaRDD<WriteStatus>> execute() {
+    try {
+      return SparkBulkInsertHelper.newInstance().bulkInsert(inputRecordsRDD, instantTime, table, config,
+          this, true, bulkInsertPartitioner);
+    } catch (HoodieInsertException ie) {
+      throw ie;
+    } catch (Throwable e) {
+      throw new HoodieInsertException("Failed to bulk insert for commit time " + instantTime, e);
+    }
+  }
+}
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/BulkInsertPreppedDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkBulkInsertPreppedDeltaCommitActionExecutor.java
similarity index 65%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/BulkInsertPreppedDeltaCommitActionExecutor.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkBulkInsertPreppedDeltaCommitActionExecutor.java
index 7c95600..21fc013 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/BulkInsertPreppedDeltaCommitActionExecutor.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkBulkInsertPreppedDeltaCommitActionExecutor.java
@@ -18,6 +18,8 @@
 
 package org.apache.hudi.table.action.deltacommit;
 
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.model.WriteOperationType;
@@ -27,30 +29,29 @@
 import org.apache.hudi.table.HoodieTable;
 import org.apache.hudi.table.BulkInsertPartitioner;
 
-import org.apache.hudi.table.action.commit.BulkInsertHelper;
 import org.apache.hudi.table.action.HoodieWriteMetadata;
+import org.apache.hudi.table.action.commit.SparkBulkInsertHelper;
 import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
 
-public class BulkInsertPreppedDeltaCommitActionExecutor<T extends HoodieRecordPayload<T>>
-    extends DeltaCommitActionExecutor<T> {
+public class SparkBulkInsertPreppedDeltaCommitActionExecutor<T extends HoodieRecordPayload<T>>
+    extends AbstractSparkDeltaCommitActionExecutor<T> {
 
   private final JavaRDD<HoodieRecord<T>> preppedInputRecordRdd;
   private final Option<BulkInsertPartitioner<T>> bulkInsertPartitioner;
 
-  public BulkInsertPreppedDeltaCommitActionExecutor(JavaSparkContext jsc,
-      HoodieWriteConfig config, HoodieTable table,
-      String instantTime, JavaRDD<HoodieRecord<T>> preppedInputRecordRdd,
-      Option<BulkInsertPartitioner<T>> bulkInsertPartitioner) {
-    super(jsc, config, table, instantTime, WriteOperationType.BULK_INSERT);
+  public SparkBulkInsertPreppedDeltaCommitActionExecutor(HoodieSparkEngineContext context,
+                                                         HoodieWriteConfig config, HoodieTable table,
+                                                         String instantTime, JavaRDD<HoodieRecord<T>> preppedInputRecordRdd,
+                                                         Option<BulkInsertPartitioner<T>> bulkInsertPartitioner) {
+    super(context, config, table, instantTime, WriteOperationType.BULK_INSERT);
     this.preppedInputRecordRdd = preppedInputRecordRdd;
     this.bulkInsertPartitioner = bulkInsertPartitioner;
   }
 
   @Override
-  public HoodieWriteMetadata execute() {
+  public HoodieWriteMetadata<JavaRDD<WriteStatus>> execute() {
     try {
-      return BulkInsertHelper.bulkInsert(preppedInputRecordRdd, instantTime, (HoodieTable<T>) table, config,
+      return SparkBulkInsertHelper.newInstance().bulkInsert(preppedInputRecordRdd, instantTime, table, config,
           this, false, bulkInsertPartitioner);
     } catch (Throwable e) {
       if (e instanceof HoodieInsertException) {
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/DeleteDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkDeleteDeltaCommitActionExecutor.java
similarity index 61%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/DeleteDeltaCommitActionExecutor.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkDeleteDeltaCommitActionExecutor.java
index 53d4d84..4fb6a90 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/DeleteDeltaCommitActionExecutor.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkDeleteDeltaCommitActionExecutor.java
@@ -18,30 +18,32 @@
 
 package org.apache.hudi.table.action.deltacommit;
 
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.model.HoodieKey;
 import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.model.WriteOperationType;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.table.HoodieTable;
 
-import org.apache.hudi.table.action.commit.DeleteHelper;
 import org.apache.hudi.table.action.HoodieWriteMetadata;
+import org.apache.hudi.table.action.commit.SparkDeleteHelper;
 import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
 
-public class DeleteDeltaCommitActionExecutor<T extends HoodieRecordPayload<T>>
-    extends DeltaCommitActionExecutor<T> {
+public class SparkDeleteDeltaCommitActionExecutor<T extends HoodieRecordPayload<T>>
+    extends AbstractSparkDeltaCommitActionExecutor<T> {
 
   private final JavaRDD<HoodieKey> keys;
 
-  public DeleteDeltaCommitActionExecutor(JavaSparkContext jsc,
-      HoodieWriteConfig config, HoodieTable table,
-      String instantTime, JavaRDD<HoodieKey> keys) {
-    super(jsc, config, table, instantTime, WriteOperationType.DELETE);
+  public SparkDeleteDeltaCommitActionExecutor(HoodieSparkEngineContext context,
+                                              HoodieWriteConfig config, HoodieTable table,
+                                              String instantTime, JavaRDD<HoodieKey> keys) {
+    super(context, config, table, instantTime, WriteOperationType.DELETE);
     this.keys = keys;
   }
 
-  public HoodieWriteMetadata execute() {
-    return DeleteHelper.execute(instantTime, keys, jsc, config, (HoodieTable<T>)table, this);
+  @Override
+  public HoodieWriteMetadata<JavaRDD<WriteStatus>> execute() {
+    return SparkDeleteHelper.newInstance().execute(instantTime, keys, context, config, table, this);
   }
 }
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/InsertDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkInsertDeltaCommitActionExecutor.java
similarity index 63%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/InsertDeltaCommitActionExecutor.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkInsertDeltaCommitActionExecutor.java
index 2124165..fcaedee 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/InsertDeltaCommitActionExecutor.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkInsertDeltaCommitActionExecutor.java
@@ -18,6 +18,8 @@
 
 package org.apache.hudi.table.action.deltacommit;
 
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.model.WriteOperationType;
@@ -25,25 +27,24 @@
 import org.apache.hudi.table.HoodieTable;
 
 import org.apache.hudi.table.action.HoodieWriteMetadata;
-import org.apache.hudi.table.action.commit.WriteHelper;
+import org.apache.hudi.table.action.commit.SparkWriteHelper;
 import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
 
-public class InsertDeltaCommitActionExecutor<T extends HoodieRecordPayload<T>>
-    extends DeltaCommitActionExecutor<T> {
+public class SparkInsertDeltaCommitActionExecutor<T extends HoodieRecordPayload<T>>
+    extends AbstractSparkDeltaCommitActionExecutor<T> {
 
   private final JavaRDD<HoodieRecord<T>> inputRecordsRDD;
 
-  public InsertDeltaCommitActionExecutor(JavaSparkContext jsc,
-      HoodieWriteConfig config, HoodieTable table,
-      String instantTime, JavaRDD<HoodieRecord<T>> inputRecordsRDD) {
-    super(jsc, config, table, instantTime, WriteOperationType.INSERT);
+  public SparkInsertDeltaCommitActionExecutor(HoodieSparkEngineContext context,
+                                              HoodieWriteConfig config, HoodieTable table,
+                                              String instantTime, JavaRDD<HoodieRecord<T>> inputRecordsRDD) {
+    super(context, config, table, instantTime, WriteOperationType.INSERT);
     this.inputRecordsRDD = inputRecordsRDD;
   }
 
   @Override
-  public HoodieWriteMetadata execute() {
-    return WriteHelper.write(instantTime, inputRecordsRDD, jsc, (HoodieTable<T>) table,
+  public HoodieWriteMetadata<JavaRDD<WriteStatus>> execute() {
+    return SparkWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table,
         config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(),this, false);
   }
 }
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/InsertPreppedDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkInsertPreppedDeltaCommitActionExecutor.java
similarity index 65%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/InsertPreppedDeltaCommitActionExecutor.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkInsertPreppedDeltaCommitActionExecutor.java
index 0fb787e..1f1e016 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/InsertPreppedDeltaCommitActionExecutor.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkInsertPreppedDeltaCommitActionExecutor.java
@@ -18,6 +18,8 @@
 
 package org.apache.hudi.table.action.deltacommit;
 
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.model.WriteOperationType;
@@ -25,21 +27,21 @@
 import org.apache.hudi.table.HoodieTable;
 import org.apache.hudi.table.action.HoodieWriteMetadata;
 import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
 
-public class InsertPreppedDeltaCommitActionExecutor<T extends HoodieRecordPayload<T>>
-    extends DeltaCommitActionExecutor<T> {
+public class SparkInsertPreppedDeltaCommitActionExecutor<T extends HoodieRecordPayload<T>>
+    extends AbstractSparkDeltaCommitActionExecutor<T> {
 
   private final JavaRDD<HoodieRecord<T>> preppedRecords;
 
-  public InsertPreppedDeltaCommitActionExecutor(JavaSparkContext jsc,
-      HoodieWriteConfig config, HoodieTable table,
-      String instantTime, JavaRDD<HoodieRecord<T>> preppedRecords) {
-    super(jsc, config, table, instantTime, WriteOperationType.INSERT_PREPPED);
+  public SparkInsertPreppedDeltaCommitActionExecutor(HoodieSparkEngineContext context,
+                                                     HoodieWriteConfig config, HoodieTable table,
+                                                     String instantTime, JavaRDD<HoodieRecord<T>> preppedRecords) {
+    super(context, config, table, instantTime, WriteOperationType.INSERT_PREPPED);
     this.preppedRecords = preppedRecords;
   }
 
-  public HoodieWriteMetadata execute() {
+  @Override
+  public HoodieWriteMetadata<JavaRDD<WriteStatus>> execute() {
     return super.execute(preppedRecords);
   }
 }
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/UpsertDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertDeltaCommitActionExecutor.java
similarity index 67%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/UpsertDeltaCommitActionExecutor.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertDeltaCommitActionExecutor.java
index 1809078..82aa081 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/UpsertDeltaCommitActionExecutor.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertDeltaCommitActionExecutor.java
@@ -18,6 +18,7 @@
 
 package org.apache.hudi.table.action.deltacommit;
 
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.model.WriteOperationType;
@@ -25,25 +26,24 @@
 import org.apache.hudi.table.HoodieTable;
 
 import org.apache.hudi.table.action.HoodieWriteMetadata;
-import org.apache.hudi.table.action.commit.WriteHelper;
+import org.apache.hudi.table.action.commit.SparkWriteHelper;
 import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
 
-public class UpsertDeltaCommitActionExecutor<T extends HoodieRecordPayload<T>>
-    extends DeltaCommitActionExecutor<T> {
+public class SparkUpsertDeltaCommitActionExecutor<T extends HoodieRecordPayload<T>>
+    extends AbstractSparkDeltaCommitActionExecutor<T> {
 
   private JavaRDD<HoodieRecord<T>> inputRecordsRDD;
 
-  public UpsertDeltaCommitActionExecutor(JavaSparkContext jsc,
-      HoodieWriteConfig config, HoodieTable table,
-      String instantTime, JavaRDD<HoodieRecord<T>> inputRecordsRDD) {
-    super(jsc, config, table, instantTime, WriteOperationType.UPSERT);
+  public SparkUpsertDeltaCommitActionExecutor(HoodieSparkEngineContext context,
+                                              HoodieWriteConfig config, HoodieTable table,
+                                              String instantTime, JavaRDD<HoodieRecord<T>> inputRecordsRDD) {
+    super(context, config, table, instantTime, WriteOperationType.UPSERT);
     this.inputRecordsRDD = inputRecordsRDD;
   }
 
   @Override
   public HoodieWriteMetadata execute() {
-    return WriteHelper.write(instantTime, inputRecordsRDD, jsc, (HoodieTable<T>) table,
+    return SparkWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table,
         config.shouldCombineBeforeUpsert(), config.getUpsertShuffleParallelism(),this, true);
   }
 }
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/UpsertDeltaCommitPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertDeltaCommitPartitioner.java
similarity index 93%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/UpsertDeltaCommitPartitioner.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertDeltaCommitPartitioner.java
index f5a4370..48a0ff0 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/UpsertDeltaCommitPartitioner.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertDeltaCommitPartitioner.java
@@ -18,6 +18,7 @@
 
 package org.apache.hudi.table.action.deltacommit;
 
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.FileSlice;
 import org.apache.hudi.common.model.HoodieLogFile;
@@ -32,7 +33,6 @@
 
 import org.apache.hudi.table.action.commit.SmallFile;
 import org.apache.hudi.table.action.commit.UpsertPartitioner;
-import org.apache.spark.api.java.JavaSparkContext;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -42,11 +42,11 @@
  * UpsertPartitioner for MergeOnRead table type, this allows auto correction of small parquet files to larger ones
  * without the need for an index in the logFile.
  */
-public class UpsertDeltaCommitPartitioner<T extends HoodieRecordPayload<T>> extends UpsertPartitioner<T> {
+public class SparkUpsertDeltaCommitPartitioner<T extends HoodieRecordPayload<T>> extends UpsertPartitioner<T> {
 
-  UpsertDeltaCommitPartitioner(WorkloadProfile profile, JavaSparkContext jsc, HoodieTable<T> table,
-      HoodieWriteConfig config) {
-    super(profile, jsc, table, config);
+  SparkUpsertDeltaCommitPartitioner(WorkloadProfile profile, HoodieSparkEngineContext context, HoodieTable table,
+                                    HoodieWriteConfig config) {
+    super(profile, context, table, config);
   }
 
   @Override
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/InsertPreppedDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertPreppedDeltaCommitActionExecutor.java
similarity index 65%
copy from hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/InsertPreppedDeltaCommitActionExecutor.java
copy to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertPreppedDeltaCommitActionExecutor.java
index 0fb787e..3509efa 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/InsertPreppedDeltaCommitActionExecutor.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertPreppedDeltaCommitActionExecutor.java
@@ -18,6 +18,8 @@
 
 package org.apache.hudi.table.action.deltacommit;
 
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.model.WriteOperationType;
@@ -25,21 +27,21 @@
 import org.apache.hudi.table.HoodieTable;
 import org.apache.hudi.table.action.HoodieWriteMetadata;
 import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
 
-public class InsertPreppedDeltaCommitActionExecutor<T extends HoodieRecordPayload<T>>
-    extends DeltaCommitActionExecutor<T> {
+public class SparkUpsertPreppedDeltaCommitActionExecutor<T extends HoodieRecordPayload<T>>
+    extends AbstractSparkDeltaCommitActionExecutor<T> {
 
   private final JavaRDD<HoodieRecord<T>> preppedRecords;
 
-  public InsertPreppedDeltaCommitActionExecutor(JavaSparkContext jsc,
-      HoodieWriteConfig config, HoodieTable table,
-      String instantTime, JavaRDD<HoodieRecord<T>> preppedRecords) {
-    super(jsc, config, table, instantTime, WriteOperationType.INSERT_PREPPED);
+  public SparkUpsertPreppedDeltaCommitActionExecutor(HoodieSparkEngineContext context,
+                                                     HoodieWriteConfig config, HoodieTable table,
+                                                     String instantTime, JavaRDD<HoodieRecord<T>> preppedRecords) {
+    super(context, config, table, instantTime, WriteOperationType.UPSERT_PREPPED);
     this.preppedRecords = preppedRecords;
   }
 
-  public HoodieWriteMetadata execute() {
+  @Override
+  public HoodieWriteMetadata<JavaRDD<WriteStatus>> execute() {
     return super.execute(preppedRecords);
   }
 }
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/restore/CopyOnWriteRestoreActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/restore/SparkCopyOnWriteRestoreActionExecutor.java
similarity index 60%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/restore/CopyOnWriteRestoreActionExecutor.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/restore/SparkCopyOnWriteRestoreActionExecutor.java
index cbd3127..101b321 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/restore/CopyOnWriteRestoreActionExecutor.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/restore/SparkCopyOnWriteRestoreActionExecutor.java
@@ -19,30 +19,38 @@
 package org.apache.hudi.table.action.restore;
 
 import org.apache.hudi.avro.model.HoodieRollbackMetadata;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
 import org.apache.hudi.common.table.timeline.HoodieInstant;
 import org.apache.hudi.common.table.timeline.HoodieTimeline;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.exception.HoodieRollbackException;
 import org.apache.hudi.table.HoodieTable;
-import org.apache.hudi.table.action.rollback.CopyOnWriteRollbackActionExecutor;
-import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.hudi.table.action.rollback.SparkCopyOnWriteRollbackActionExecutor;
 
-public class CopyOnWriteRestoreActionExecutor extends BaseRestoreActionExecutor {
+import org.apache.spark.api.java.JavaRDD;
 
-  public CopyOnWriteRestoreActionExecutor(JavaSparkContext jsc,
-                                          HoodieWriteConfig config,
-                                          HoodieTable<?> table,
-                                          String instantTime,
-                                          String restoreInstantTime) {
-    super(jsc, config, table, instantTime, restoreInstantTime);
+@SuppressWarnings("checkstyle:LineLength")
+public class SparkCopyOnWriteRestoreActionExecutor<T extends HoodieRecordPayload> extends
+    BaseRestoreActionExecutor<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
+
+  public SparkCopyOnWriteRestoreActionExecutor(HoodieSparkEngineContext context,
+                                               HoodieWriteConfig config,
+                                               HoodieTable table,
+                                               String instantTime,
+                                               String restoreInstantTime) {
+    super(context, config, table, instantTime, restoreInstantTime);
   }
 
   @Override
   protected HoodieRollbackMetadata rollbackInstant(HoodieInstant instantToRollback) {
     table.getMetaClient().reloadActiveTimeline();
-    CopyOnWriteRollbackActionExecutor rollbackActionExecutor = new CopyOnWriteRollbackActionExecutor(
-        jsc,
+    SparkCopyOnWriteRollbackActionExecutor rollbackActionExecutor = new SparkCopyOnWriteRollbackActionExecutor(
+        (HoodieSparkEngineContext) context,
         config,
         table,
         HoodieActiveTimeline.createNewInstantTime(),
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/restore/MergeOnReadRestoreActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/restore/SparkMergeOnReadRestoreActionExecutor.java
similarity index 64%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/restore/MergeOnReadRestoreActionExecutor.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/restore/SparkMergeOnReadRestoreActionExecutor.java
index edb9acd..c320579 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/restore/MergeOnReadRestoreActionExecutor.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/restore/SparkMergeOnReadRestoreActionExecutor.java
@@ -19,29 +19,37 @@
 package org.apache.hudi.table.action.restore;
 
 import org.apache.hudi.avro.model.HoodieRollbackMetadata;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
 import org.apache.hudi.common.table.timeline.HoodieInstant;
 import org.apache.hudi.common.table.timeline.HoodieTimeline;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.table.HoodieTable;
-import org.apache.hudi.table.action.rollback.MergeOnReadRollbackActionExecutor;
-import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.hudi.table.action.rollback.SparkMergeOnReadRollbackActionExecutor;
 
-public class MergeOnReadRestoreActionExecutor extends BaseRestoreActionExecutor {
+import org.apache.spark.api.java.JavaRDD;
 
-  public MergeOnReadRestoreActionExecutor(JavaSparkContext jsc,
-                                          HoodieWriteConfig config,
-                                          HoodieTable<?> table,
-                                          String instantTime,
-                                          String restoreInstantTime) {
-    super(jsc, config, table, instantTime, restoreInstantTime);
+@SuppressWarnings("checkstyle:LineLength")
+public class SparkMergeOnReadRestoreActionExecutor<T extends HoodieRecordPayload> extends
+    BaseRestoreActionExecutor<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
+
+  public SparkMergeOnReadRestoreActionExecutor(HoodieSparkEngineContext context,
+                                               HoodieWriteConfig config,
+                                               HoodieTable table,
+                                               String instantTime,
+                                               String restoreInstantTime) {
+    super(context, config, table, instantTime, restoreInstantTime);
   }
 
   @Override
   protected HoodieRollbackMetadata rollbackInstant(HoodieInstant instantToRollback) {
     table.getMetaClient().reloadActiveTimeline();
-    MergeOnReadRollbackActionExecutor rollbackActionExecutor = new MergeOnReadRollbackActionExecutor(
-        jsc,
+    SparkMergeOnReadRollbackActionExecutor rollbackActionExecutor = new SparkMergeOnReadRollbackActionExecutor(
+        context,
         config,
         table,
         HoodieActiveTimeline.createNewInstantTime(),
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackHelper.java
similarity index 90%
rename from hudi-client/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackHelper.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackHelper.java
index 3c94df4..9cf2434 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackHelper.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackHelper.java
@@ -18,7 +18,9 @@
 
 package org.apache.hudi.table.action.rollback;
 
+import org.apache.hudi.client.common.HoodieEngineContext;
 import org.apache.hudi.common.HoodieRollbackStat;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.HoodieLogFile;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
@@ -68,35 +70,36 @@
   /**
    * Performs all rollback actions that we have collected in parallel.
    */
-  public List<HoodieRollbackStat> performRollback(JavaSparkContext jsc, HoodieInstant instantToRollback, List<ListingBasedRollbackRequest> rollbackRequests) {
+  public List<HoodieRollbackStat> performRollback(HoodieEngineContext context, HoodieInstant instantToRollback, List<ListingBasedRollbackRequest> rollbackRequests) {
     int sparkPartitions = Math.max(Math.min(rollbackRequests.size(), config.getRollbackParallelism()), 1);
-    jsc.setJobGroup(this.getClass().getSimpleName(), "Perform rollback actions");
-    JavaPairRDD<String, HoodieRollbackStat> partitionPathRollbackStatsPairRDD = maybeDeleteAndCollectStats(jsc, instantToRollback, rollbackRequests, sparkPartitions, true);
+    context.setJobStatus(this.getClass().getSimpleName(), "Perform rollback actions");
+    JavaPairRDD<String, HoodieRollbackStat> partitionPathRollbackStatsPairRDD = maybeDeleteAndCollectStats(context, instantToRollback, rollbackRequests, sparkPartitions, true);
     return partitionPathRollbackStatsPairRDD.reduceByKey(RollbackUtils::mergeRollbackStat).map(Tuple2::_2).collect();
   }
 
   /**
    * Collect all file info that needs to be rollbacked.
    */
-  public List<HoodieRollbackStat> collectRollbackStats(JavaSparkContext jsc, HoodieInstant instantToRollback, List<ListingBasedRollbackRequest> rollbackRequests) {
+  public List<HoodieRollbackStat> collectRollbackStats(HoodieEngineContext context, HoodieInstant instantToRollback, List<ListingBasedRollbackRequest> rollbackRequests) {
     int sparkPartitions = Math.max(Math.min(rollbackRequests.size(), config.getRollbackParallelism()), 1);
-    jsc.setJobGroup(this.getClass().getSimpleName(), "Collect rollback stats for upgrade/downgrade");
-    JavaPairRDD<String, HoodieRollbackStat> partitionPathRollbackStatsPairRDD = maybeDeleteAndCollectStats(jsc, instantToRollback, rollbackRequests, sparkPartitions, false);
+    context.setJobStatus(this.getClass().getSimpleName(), "Collect rollback stats for upgrade/downgrade");
+    JavaPairRDD<String, HoodieRollbackStat> partitionPathRollbackStatsPairRDD = maybeDeleteAndCollectStats(context, instantToRollback, rollbackRequests, sparkPartitions, false);
     return partitionPathRollbackStatsPairRDD.map(Tuple2::_2).collect();
   }
 
   /**
    * May be delete interested files and collect stats or collect stats only.
    *
-   * @param jsc instance of {@link JavaSparkContext} to use.
+   * @param context instance of {@link HoodieEngineContext} to use.
    * @param instantToRollback {@link HoodieInstant} of interest for which deletion or collect stats is requested.
    * @param rollbackRequests List of {@link ListingBasedRollbackRequest} to be operated on.
    * @param sparkPartitions number of spark partitions to use for parallelism.
    * @param doDelete {@code true} if deletion has to be done. {@code false} if only stats are to be collected w/o performing any deletes.
    * @return stats collected with or w/o actual deletions.
    */
-  JavaPairRDD<String, HoodieRollbackStat> maybeDeleteAndCollectStats(JavaSparkContext jsc, HoodieInstant instantToRollback, List<ListingBasedRollbackRequest> rollbackRequests,
+  JavaPairRDD<String, HoodieRollbackStat> maybeDeleteAndCollectStats(HoodieEngineContext context, HoodieInstant instantToRollback, List<ListingBasedRollbackRequest> rollbackRequests,
                                                                      int sparkPartitions, boolean doDelete) {
+    JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context);
     return jsc.parallelize(rollbackRequests, sparkPartitions).mapToPair(rollbackRequest -> {
       switch (rollbackRequest.getType()) {
         case DELETE_DATA_FILES_ONLY: {
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/rollback/SparkCopyOnWriteRollbackActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/rollback/SparkCopyOnWriteRollbackActionExecutor.java
new file mode 100644
index 0000000..965d805
--- /dev/null
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/rollback/SparkCopyOnWriteRollbackActionExecutor.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.table.action.rollback;
+
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
+import org.apache.hudi.common.HoodieRollbackStat;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.table.HoodieTable;
+
+import org.apache.spark.api.java.JavaRDD;
+
+import java.util.List;
+
+@SuppressWarnings("checkstyle:LineLength")
+public class SparkCopyOnWriteRollbackActionExecutor<T extends HoodieRecordPayload> extends
+    BaseCopyOnWriteRollbackActionExecutor<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
+  public SparkCopyOnWriteRollbackActionExecutor(HoodieSparkEngineContext context,
+                                                HoodieWriteConfig config,
+                                                HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table,
+                                                String instantTime,
+                                                HoodieInstant commitInstant,
+                                                boolean deleteInstants) {
+    super(context, config, table, instantTime, commitInstant, deleteInstants);
+  }
+
+  public SparkCopyOnWriteRollbackActionExecutor(HoodieSparkEngineContext context,
+                                                HoodieWriteConfig config,
+                                                HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table,
+                                                String instantTime,
+                                                HoodieInstant commitInstant,
+                                                boolean deleteInstants,
+                                                boolean skipTimelinePublish,
+                                                boolean useMarkerBasedStrategy) {
+    super(context, config, table, instantTime, commitInstant, deleteInstants, skipTimelinePublish, useMarkerBasedStrategy);
+  }
+
+  @Override
+  protected BaseRollbackActionExecutor.RollbackStrategy getRollbackStrategy() {
+    if (useMarkerBasedStrategy) {
+      return new SparkMarkerBasedRollbackStrategy(table, context, config, instantTime);
+    } else {
+      return this::executeRollbackUsingFileListing;
+    }
+  }
+
+  @Override
+  protected List<HoodieRollbackStat> executeRollbackUsingFileListing(HoodieInstant instantToRollback) {
+    List<ListingBasedRollbackRequest> rollbackRequests = RollbackUtils.generateRollbackRequestsByListingCOW(table.getMetaClient().getFs(), table.getMetaClient().getBasePath(),
+        config.shouldAssumeDatePartitioning());
+    return new ListingBasedRollbackHelper(table.getMetaClient(), config).performRollback(context, instantToRollback, rollbackRequests);
+  }
+}
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/rollback/SparkMarkerBasedRollbackStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/rollback/SparkMarkerBasedRollbackStrategy.java
new file mode 100644
index 0000000..065b22d
--- /dev/null
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/rollback/SparkMarkerBasedRollbackStrategy.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.hudi.table.action.rollback;
+
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
+import org.apache.hudi.common.HoodieRollbackStat;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.model.IOType;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieRollbackException;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.table.MarkerFiles;
+
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+
+import java.util.List;
+
+import scala.Tuple2;
+
+@SuppressWarnings("checkstyle:LineLength")
+public class SparkMarkerBasedRollbackStrategy<T extends HoodieRecordPayload> extends AbstractMarkerBasedRollbackStrategy<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
+  public SparkMarkerBasedRollbackStrategy(HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table, HoodieEngineContext context, HoodieWriteConfig config, String instantTime) {
+    super(table, context, config, instantTime);
+  }
+
+  @Override
+  public List<HoodieRollbackStat> execute(HoodieInstant instantToRollback) {
+    JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context);
+    try {
+      MarkerFiles markerFiles = new MarkerFiles(table, instantToRollback.getTimestamp());
+      List<String> markerFilePaths = markerFiles.allMarkerFilePaths();
+      int parallelism = Math.max(Math.min(markerFilePaths.size(), config.getRollbackParallelism()), 1);
+      return jsc.parallelize(markerFilePaths, parallelism)
+          .map(markerFilePath -> {
+            String typeStr = markerFilePath.substring(markerFilePath.lastIndexOf(".") + 1);
+            IOType type = IOType.valueOf(typeStr);
+            switch (type) {
+              case MERGE:
+                return undoMerge(MarkerFiles.stripMarkerSuffix(markerFilePath));
+              case APPEND:
+                return undoAppend(MarkerFiles.stripMarkerSuffix(markerFilePath), instantToRollback);
+              case CREATE:
+                return undoCreate(MarkerFiles.stripMarkerSuffix(markerFilePath));
+              default:
+                throw new HoodieRollbackException("Unknown marker type, during rollback of " + instantToRollback);
+            }
+          })
+          .mapToPair(rollbackStat -> new Tuple2<>(rollbackStat.getPartitionPath(), rollbackStat))
+          .reduceByKey(RollbackUtils::mergeRollbackStat)
+          .map(Tuple2::_2).collect();
+    } catch (Exception e) {
+      throw new HoodieRollbackException("Error rolling back using marker files written for " + instantToRollback, e);
+    }
+  }
+}
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/rollback/SparkMergeOnReadRollbackActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/rollback/SparkMergeOnReadRollbackActionExecutor.java
new file mode 100644
index 0000000..459ab12
--- /dev/null
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/rollback/SparkMergeOnReadRollbackActionExecutor.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.hudi.table.action.rollback;
+
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
+import org.apache.hudi.common.HoodieRollbackStat;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.table.HoodieTable;
+
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+
+import java.io.IOException;
+import java.util.List;
+
+@SuppressWarnings("checkstyle:LineLength")
+public class SparkMergeOnReadRollbackActionExecutor<T extends HoodieRecordPayload> extends
+    BaseMergeOnReadRollbackActionExecutor<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
+  public SparkMergeOnReadRollbackActionExecutor(HoodieEngineContext context,
+                                                HoodieWriteConfig config,
+                                                HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table,
+                                                String instantTime,
+                                                HoodieInstant commitInstant,
+                                                boolean deleteInstants) {
+    super(context, config, table, instantTime, commitInstant, deleteInstants);
+  }
+
+  public SparkMergeOnReadRollbackActionExecutor(HoodieEngineContext context,
+                                                HoodieWriteConfig config,
+                                                HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table,
+                                                String instantTime,
+                                                HoodieInstant commitInstant,
+                                                boolean deleteInstants,
+                                                boolean skipTimelinePublish,
+                                                boolean useMarkerBasedStrategy) {
+    super(context, config, table, instantTime, commitInstant, deleteInstants, skipTimelinePublish, useMarkerBasedStrategy);
+  }
+
+  @Override
+  protected BaseRollbackActionExecutor.RollbackStrategy getRollbackStrategy() {
+    if (useMarkerBasedStrategy) {
+      return new SparkMarkerBasedRollbackStrategy(table, context, config, instantTime);
+    } else {
+      return this::executeRollbackUsingFileListing;
+    }
+  }
+
+  @Override
+  protected List<HoodieRollbackStat> executeRollbackUsingFileListing(HoodieInstant resolvedInstant) {
+    List<ListingBasedRollbackRequest> rollbackRequests;
+    JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context);
+    try {
+      rollbackRequests = RollbackUtils.generateRollbackRequestsUsingFileListingMOR(resolvedInstant, table, context);
+    } catch (IOException e) {
+      throw new HoodieIOException("Error generating rollback requests by file listing.", e);
+    }
+    return new ListingBasedRollbackHelper(table.getMetaClient(), config).performRollback(context, resolvedInstant, rollbackRequests);
+  }
+}
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/upgrade/OneToZeroDowngradeHandler.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/OneToZeroDowngradeHandler.java
similarity index 79%
rename from hudi-client/src/main/java/org/apache/hudi/table/upgrade/OneToZeroDowngradeHandler.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/OneToZeroDowngradeHandler.java
index 108bdbb..52849cb 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/upgrade/OneToZeroDowngradeHandler.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/OneToZeroDowngradeHandler.java
@@ -18,32 +18,31 @@
 
 package org.apache.hudi.table.upgrade;
 
+import org.apache.hudi.client.common.HoodieEngineContext;
 import org.apache.hudi.common.table.timeline.HoodieInstant;
 import org.apache.hudi.common.table.timeline.HoodieTimeline;
 import org.apache.hudi.config.HoodieWriteConfig;
-import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.table.HoodieSparkTable;
 import org.apache.hudi.table.MarkerFiles;
 
-import org.apache.spark.api.java.JavaSparkContext;
-
 import java.util.List;
 import java.util.stream.Collectors;
 
 /**
  * Downgrade handle to assist in downgrading hoodie table from version 1 to 0.
  */
-public class OneToZeroDowngradeHandler implements DowngradeHandler {
+public  class OneToZeroDowngradeHandler implements DowngradeHandler {
 
   @Override
-  public void downgrade(HoodieWriteConfig config, JavaSparkContext jsc, String instantTime) {
+  public void downgrade(HoodieWriteConfig config, HoodieEngineContext context, String instantTime) {
     // fetch pending commit info
-    HoodieTable table = HoodieTable.create(config, jsc.hadoopConfiguration());
+    HoodieSparkTable table = HoodieSparkTable.create(config, context);
     HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompaction();
     List<HoodieInstant> commits = inflightTimeline.getReverseOrderedInstants().collect(Collectors.toList());
     for (HoodieInstant commitInstant : commits) {
       // delete existing marker files
       MarkerFiles markerFiles = new MarkerFiles(table, commitInstant.getTimestamp());
-      markerFiles.quietDeleteMarkerDir(jsc, config.getMarkersDeleteParallelism());
+      markerFiles.quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism());
     }
   }
 }
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/SparkUpgradeDowngrade.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/SparkUpgradeDowngrade.java
new file mode 100644
index 0000000..9c13c5a
--- /dev/null
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/SparkUpgradeDowngrade.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.hudi.table.upgrade;
+
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.HoodieTableVersion;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieUpgradeDowngradeException;
+
+import java.io.IOException;
+
+public class SparkUpgradeDowngrade extends AbstractUpgradeDowngrade {
+
+  public SparkUpgradeDowngrade(HoodieTableMetaClient metaClient, HoodieWriteConfig config, HoodieEngineContext context) {
+    super(metaClient, config, context);
+  }
+
+  @Override
+  public void run(HoodieTableMetaClient metaClient,
+                  HoodieTableVersion toVersion,
+                  HoodieWriteConfig config,
+                  HoodieEngineContext context,
+                  String instantTime) {
+    try {
+      new SparkUpgradeDowngrade(metaClient, config, context).run(toVersion, instantTime);
+    } catch (IOException e) {
+      throw new HoodieUpgradeDowngradeException("Error during upgrade/downgrade to version:" + toVersion, e);
+    }
+
+  }
+
+  @Override
+  protected void upgrade(HoodieTableVersion fromVersion, HoodieTableVersion toVersion, String instantTime) {
+    if (fromVersion == HoodieTableVersion.ZERO && toVersion == HoodieTableVersion.ONE) {
+      new ZeroToOneUpgradeHandler().upgrade(config, context, instantTime);
+    } else {
+      throw new HoodieUpgradeDowngradeException(fromVersion.versionCode(), toVersion.versionCode(), true);
+    }
+  }
+
+  @Override
+  protected void downgrade(HoodieTableVersion fromVersion, HoodieTableVersion toVersion, String instantTime) {
+    if (fromVersion == HoodieTableVersion.ONE && toVersion == HoodieTableVersion.ZERO) {
+      new OneToZeroDowngradeHandler().downgrade(config, context, instantTime);
+    } else {
+      throw new HoodieUpgradeDowngradeException(fromVersion.versionCode(), toVersion.versionCode(), false);
+    }
+  }
+}
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java
similarity index 86%
rename from hudi-client/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java
rename to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java
index e9c9e28..7e3faf3 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/upgrade/ZeroToOneUpgradeHandler.java
@@ -18,6 +18,7 @@
 
 package org.apache.hudi.table.upgrade;
 
+import org.apache.hudi.client.common.HoodieEngineContext;
 import org.apache.hudi.common.HoodieRollbackStat;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.HoodieTableType;
@@ -28,15 +29,15 @@
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.exception.HoodieRollbackException;
+import org.apache.hudi.table.HoodieSparkTable;
 import org.apache.hudi.table.HoodieTable;
 import org.apache.hudi.table.MarkerFiles;
 import org.apache.hudi.table.action.rollback.ListingBasedRollbackHelper;
 import org.apache.hudi.table.action.rollback.ListingBasedRollbackRequest;
-import org.apache.hudi.table.action.rollback.RollbackUtils;
 
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
-import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.hudi.table.action.rollback.RollbackUtils;
 
 import java.util.List;
 import java.util.stream.Collectors;
@@ -47,9 +48,9 @@
 public class ZeroToOneUpgradeHandler implements UpgradeHandler {
 
   @Override
-  public void upgrade(HoodieWriteConfig config, JavaSparkContext jsc, String instantTime) {
+  public void upgrade(HoodieWriteConfig config, HoodieEngineContext context, String instantTime) {
     // fetch pending commit info
-    HoodieTable table = HoodieTable.create(config, jsc.hadoopConfiguration());
+    HoodieSparkTable table = HoodieSparkTable.create(config, context);
     HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompaction();
     List<String> commits = inflightTimeline.getReverseOrderedInstants().map(HoodieInstant::getTimestamp)
         .collect(Collectors.toList());
@@ -59,7 +60,7 @@
     }
     for (String commit : commits) {
       // for every pending commit, delete old marker files and re-create marker files in new format
-      recreateMarkerFiles(commit, table, jsc, config.getMarkersDeleteParallelism());
+      recreateMarkerFiles(commit, table, context, config.getMarkersDeleteParallelism());
     }
   }
 
@@ -70,11 +71,14 @@
    * Step3: recreate marker files for all interested files.
    *
    * @param commitInstantTime instant of interest for which marker files need to be recreated.
-   * @param table instance of {@link HoodieTable} to use
-   * @param jsc instance of {@link JavaSparkContext} to use
+   * @param table instance of {@link HoodieSparkTable} to use
+   * @param context instance of {@link HoodieEngineContext} to use
    * @throws HoodieRollbackException on any exception during upgrade.
    */
-  private static void recreateMarkerFiles(final String commitInstantTime, HoodieTable table, JavaSparkContext jsc, int parallelism) throws HoodieRollbackException {
+  private static void recreateMarkerFiles(final String commitInstantTime,
+                                          HoodieSparkTable table,
+                                          HoodieEngineContext context,
+                                          int parallelism) throws HoodieRollbackException {
     try {
       // fetch hoodie instant
       Option<HoodieInstant> commitInstantOpt = Option.fromJavaOptional(table.getActiveTimeline().getCommitsTimeline().getInstants()
@@ -83,7 +87,7 @@
       if (commitInstantOpt.isPresent()) {
         // delete existing marker files
         MarkerFiles markerFiles = new MarkerFiles(table, commitInstantTime);
-        markerFiles.quietDeleteMarkerDir(jsc, parallelism);
+        markerFiles.quietDeleteMarkerDir(context, parallelism);
 
         // generate rollback stats
         List<ListingBasedRollbackRequest> rollbackRequests;
@@ -91,10 +95,10 @@
           rollbackRequests = RollbackUtils.generateRollbackRequestsByListingCOW(table.getMetaClient().getFs(), table.getMetaClient().getBasePath(),
               table.getConfig().shouldAssumeDatePartitioning());
         } else {
-          rollbackRequests = RollbackUtils.generateRollbackRequestsUsingFileListingMOR(commitInstantOpt.get(), table, jsc);
+          rollbackRequests = RollbackUtils.generateRollbackRequestsUsingFileListingMOR(commitInstantOpt.get(), table, context);
         }
         List<HoodieRollbackStat> rollbackStats = new ListingBasedRollbackHelper(table.getMetaClient(), table.getConfig())
-            .collectRollbackStats(jsc, commitInstantOpt.get(), rollbackRequests);
+            .collectRollbackStats(context, commitInstantOpt.get(), rollbackRequests);
 
         // recreate marker files adhering to marker based rollback
         for (HoodieRollbackStat rollbackStat : rollbackStats) {
diff --git a/hudi-client/src/main/resources/log4j.properties b/hudi-client/hudi-spark-client/src/main/resources/log4j.properties
similarity index 100%
rename from hudi-client/src/main/resources/log4j.properties
rename to hudi-client/hudi-spark-client/src/main/resources/log4j.properties
diff --git a/hudi-client/src/test/java/org/apache/hudi/client/TestClientRollback.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestClientRollback.java
similarity index 95%
rename from hudi-client/src/test/java/org/apache/hudi/client/TestClientRollback.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestClientRollback.java
index da4c002..d04a2df 100644
--- a/hudi-client/src/test/java/org/apache/hudi/client/TestClientRollback.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestClientRollback.java
@@ -32,7 +32,7 @@
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.exception.HoodieRollbackException;
 import org.apache.hudi.index.HoodieIndex;
-import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.table.HoodieSparkTable;
 import org.apache.hudi.testutils.HoodieClientTestBase;
 
 import org.apache.spark.api.java.JavaRDD;
@@ -61,7 +61,7 @@
   public void testSavepointAndRollback() throws Exception {
     HoodieWriteConfig cfg = getConfigBuilder().withCompactionConfig(HoodieCompactionConfig.newBuilder()
         .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(1).build()).build();
-    try (HoodieWriteClient client = getHoodieWriteClient(cfg)) {
+    try (SparkRDDWriteClient client = getHoodieWriteClient(cfg)) {
       HoodieTestDataGenerator.writePartitionMetadata(fs, HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, basePath);
 
       /**
@@ -102,7 +102,7 @@
       List<String> partitionPaths =
           FSUtils.getAllPartitionPaths(fs, cfg.getBasePath(), getConfig().shouldAssumeDatePartitioning());
       metaClient = HoodieTableMetaClient.reload(metaClient);
-      HoodieTable table = HoodieTable.create(metaClient, getConfig(), hadoopConf);
+      HoodieSparkTable table = HoodieSparkTable.create(getConfig(), context, metaClient);
       final BaseFileOnlyView view1 = table.getBaseFileOnlyView();
 
       List<HoodieBaseFile> dataFiles = partitionPaths.stream().flatMap(s -> {
@@ -127,7 +127,7 @@
       assertNoWriteErrors(statuses);
 
       metaClient = HoodieTableMetaClient.reload(metaClient);
-      table = HoodieTable.create(metaClient, getConfig(), hadoopConf);
+      table = HoodieSparkTable.create(getConfig(), context, metaClient);
       final BaseFileOnlyView view2 = table.getBaseFileOnlyView();
 
       dataFiles = partitionPaths.stream().flatMap(s -> view2.getAllBaseFiles(s).filter(f -> f.getCommitTime().equals("004"))).collect(Collectors.toList());
@@ -143,7 +143,7 @@
       client.restoreToSavepoint(savepoint.getTimestamp());
 
       metaClient = HoodieTableMetaClient.reload(metaClient);
-      table = HoodieTable.create(metaClient, getConfig(), hadoopConf);
+      table = HoodieSparkTable.create(getConfig(), context, metaClient);
       final BaseFileOnlyView view3 = table.getBaseFileOnlyView();
       dataFiles = partitionPaths.stream().flatMap(s -> view3.getAllBaseFiles(s).filter(f -> f.getCommitTime().equals("002"))).collect(Collectors.toList());
       assertEquals(3, dataFiles.size(), "The data files for commit 002 be available");
@@ -201,7 +201,7 @@
     HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
         .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build();
 
-    try (HoodieWriteClient client = getHoodieWriteClient(config, false)) {
+    try (SparkRDDWriteClient client = getHoodieWriteClient(config, false)) {
 
       // Rollback commit 1 (this should fail, since commit2 is still around)
       assertThrows(HoodieRollbackException.class, () -> {
@@ -294,7 +294,7 @@
         .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build();
 
     final String commitTime4 = "20160506030621";
-    try (HoodieWriteClient client = getHoodieWriteClient(config, false)) {
+    try (SparkRDDWriteClient client = getHoodieWriteClient(config, false)) {
       client.startCommitWithTime(commitTime4);
       // Check results, nothing changed
       assertTrue(testTable.commitExists(commitTime1));
@@ -307,7 +307,7 @@
 
     // Turn auto rollback on
     final String commitTime5 = "20160506030631";
-    try (HoodieWriteClient client = getHoodieWriteClient(config, true)) {
+    try (SparkRDDWriteClient client = getHoodieWriteClient(config, true)) {
       client.startCommitWithTime(commitTime5);
       assertTrue(testTable.commitExists(commitTime1));
       assertFalse(testTable.inflightCommitExists(commitTime2));
diff --git a/hudi-client/src/test/java/org/apache/hudi/client/TestCompactionAdminClient.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestCompactionAdminClient.java
similarity index 99%
rename from hudi-client/src/test/java/org/apache/hudi/client/TestCompactionAdminClient.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestCompactionAdminClient.java
index 1200f67..03328dd 100644
--- a/hudi-client/src/test/java/org/apache/hudi/client/TestCompactionAdminClient.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestCompactionAdminClient.java
@@ -67,7 +67,7 @@
     initPath();
     initSparkContexts();
     metaClient = HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath, MERGE_ON_READ);
-    client = new CompactionAdminClient(jsc, basePath);
+    client = new CompactionAdminClient(context, basePath);
   }
 
   @Test
diff --git a/hudi-client/src/test/java/org/apache/hudi/client/TestHoodieClientOnCopyOnWriteStorage.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientOnCopyOnWriteStorage.java
similarity index 94%
rename from hudi-client/src/test/java/org/apache/hudi/client/TestHoodieClientOnCopyOnWriteStorage.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientOnCopyOnWriteStorage.java
index 47edbbd..bbb4048 100644
--- a/hudi-client/src/test/java/org/apache/hudi/client/TestHoodieClientOnCopyOnWriteStorage.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientOnCopyOnWriteStorage.java
@@ -51,9 +51,10 @@
 import org.apache.hudi.exception.HoodieRollbackException;
 import org.apache.hudi.index.HoodieIndex;
 import org.apache.hudi.index.HoodieIndex.IndexType;
+import org.apache.hudi.table.HoodieSparkTable;
 import org.apache.hudi.table.HoodieTable;
 import org.apache.hudi.table.MarkerFiles;
-import org.apache.hudi.table.action.commit.WriteHelper;
+import org.apache.hudi.table.action.commit.SparkWriteHelper;
 import org.apache.hudi.testutils.HoodieClientTestBase;
 import org.apache.hudi.testutils.HoodieClientTestUtils;
 import org.apache.hudi.testutils.HoodieWriteableTestTable;
@@ -115,7 +116,7 @@
    */
   @Test
   public void testAutoCommitOnInsert() throws Exception {
-    testAutoCommit(HoodieWriteClient::insert, false);
+    testAutoCommit(SparkRDDWriteClient::insert, false);
   }
 
   /**
@@ -123,7 +124,7 @@
    */
   @Test
   public void testAutoCommitOnInsertPrepped() throws Exception {
-    testAutoCommit(HoodieWriteClient::insertPreppedRecords, true);
+    testAutoCommit(SparkRDDWriteClient::insertPreppedRecords, true);
   }
 
   /**
@@ -131,7 +132,7 @@
    */
   @Test
   public void testAutoCommitOnUpsert() throws Exception {
-    testAutoCommit(HoodieWriteClient::upsert, false);
+    testAutoCommit(SparkRDDWriteClient::upsert, false);
   }
 
   /**
@@ -139,7 +140,7 @@
    */
   @Test
   public void testAutoCommitOnUpsertPrepped() throws Exception {
-    testAutoCommit(HoodieWriteClient::upsertPreppedRecords, true);
+    testAutoCommit(SparkRDDWriteClient::upsertPreppedRecords, true);
   }
 
   /**
@@ -147,7 +148,7 @@
    */
   @Test
   public void testAutoCommitOnBulkInsert() throws Exception {
-    testAutoCommit(HoodieWriteClient::bulkInsert, false);
+    testAutoCommit(SparkRDDWriteClient::bulkInsert, false);
   }
 
   /**
@@ -165,11 +166,11 @@
    * @param writeFn One of HoodieWriteClient Write API
    * @throws Exception in case of failure
    */
-  private void testAutoCommit(Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieRecord>, String> writeFn,
+  private void testAutoCommit(Function3<JavaRDD<WriteStatus>, SparkRDDWriteClient, JavaRDD<HoodieRecord>, String> writeFn,
       boolean isPrepped) throws Exception {
     // Set autoCommit false
     HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build();
-    try (HoodieWriteClient client = getHoodieWriteClient(cfg);) {
+    try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) {
 
       String prevCommitTime = "000";
       String newCommitTime = "001";
@@ -190,7 +191,7 @@
    */
   @Test
   public void testDeduplicationOnInsert() throws Exception {
-    testDeduplication(HoodieWriteClient::insert);
+    testDeduplication(SparkRDDWriteClient::insert);
   }
 
   /**
@@ -198,7 +199,7 @@
    */
   @Test
   public void testDeduplicationOnBulkInsert() throws Exception {
-    testDeduplication(HoodieWriteClient::bulkInsert);
+    testDeduplication(SparkRDDWriteClient::bulkInsert);
   }
 
   /**
@@ -206,7 +207,7 @@
    */
   @Test
   public void testDeduplicationOnUpsert() throws Exception {
-    testDeduplication(HoodieWriteClient::upsert);
+    testDeduplication(SparkRDDWriteClient::upsert);
   }
 
   /**
@@ -216,7 +217,7 @@
    * @throws Exception in case of failure
    */
   private void testDeduplication(
-      Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieRecord>, String> writeFn) throws Exception {
+      Function3<JavaRDD<WriteStatus>, SparkRDDWriteClient, JavaRDD<HoodieRecord>, String> writeFn) throws Exception {
     String newCommitTime = "001";
 
     String recordKey = UUID.randomUUID().toString();
@@ -238,20 +239,20 @@
     // Global dedup should be done based on recordKey only
     HoodieIndex index = mock(HoodieIndex.class);
     when(index.isGlobal()).thenReturn(true);
-    List<HoodieRecord<RawTripTestPayload>> dedupedRecs = WriteHelper.deduplicateRecords(records, index, 1).collect();
+    List<HoodieRecord<RawTripTestPayload>> dedupedRecs = SparkWriteHelper.newInstance().deduplicateRecords(records, index, 1).collect();
     assertEquals(1, dedupedRecs.size());
     assertNodupesWithinPartition(dedupedRecs);
 
     // non-Global dedup should be done based on both recordKey and partitionPath
     index = mock(HoodieIndex.class);
     when(index.isGlobal()).thenReturn(false);
-    dedupedRecs = WriteHelper.deduplicateRecords(records, index, 1).collect();
+    dedupedRecs = SparkWriteHelper.newInstance().deduplicateRecords(records, index, 1).collect();
     assertEquals(2, dedupedRecs.size());
     assertNodupesWithinPartition(dedupedRecs);
 
     // Perform write-action and check
     JavaRDD<HoodieRecord> recordList = jsc.parallelize(Arrays.asList(recordOne, recordTwo, recordThree), 1);
-    try (HoodieWriteClient client = getHoodieWriteClient(getConfigBuilder().combineInput(true, true).build(), false);) {
+    try (SparkRDDWriteClient client = getHoodieWriteClient(getConfigBuilder().combineInput(true, true).build(), false);) {
       client.startCommitWithTime(newCommitTime);
       List<WriteStatus> statuses = writeFn.apply(client, recordList, newCommitTime).collect();
       assertNoWriteErrors(statuses);
@@ -284,7 +285,7 @@
    */
   @Test
   public void testUpserts() throws Exception {
-    testUpsertsInternal(getConfig(), HoodieWriteClient::upsert, false);
+    testUpsertsInternal(getConfig(), SparkRDDWriteClient::upsert, false);
   }
 
   /**
@@ -292,7 +293,7 @@
    */
   @Test
   public void testUpsertsPrepped() throws Exception {
-    testUpsertsInternal(getConfig(), HoodieWriteClient::upsertPreppedRecords, true);
+    testUpsertsInternal(getConfig(), SparkRDDWriteClient::upsertPreppedRecords, true);
   }
 
   /**
@@ -303,7 +304,7 @@
    * @throws Exception in case of error
    */
   private void testUpsertsInternal(HoodieWriteConfig config,
-      Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieRecord>, String> writeFn, boolean isPrepped)
+      Function3<JavaRDD<WriteStatus>, SparkRDDWriteClient, JavaRDD<HoodieRecord>, String> writeFn, boolean isPrepped)
       throws Exception {
     // Force using older timeline layout
     HoodieWriteConfig hoodieWriteConfig = getConfigBuilder().withProps(config.getProps()).withTimelineLayoutVersion(
@@ -311,13 +312,13 @@
     HoodieTableMetaClient.initTableType(metaClient.getHadoopConf(), metaClient.getBasePath(), metaClient.getTableType(),
         metaClient.getTableConfig().getTableName(), metaClient.getArchivePath(),
         metaClient.getTableConfig().getPayloadClass(), VERSION_0);
-    HoodieWriteClient client = getHoodieWriteClient(hoodieWriteConfig, false);
+    SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig, false);
 
     // Write 1 (only inserts)
     String newCommitTime = "001";
     String initCommitTime = "000";
     int numRecords = 200;
-    insertFirstBatch(hoodieWriteConfig, client, newCommitTime, initCommitTime, numRecords, HoodieWriteClient::insert,
+    insertFirstBatch(hoodieWriteConfig, client, newCommitTime, initCommitTime, numRecords, SparkRDDWriteClient::insert,
         isPrepped, true, numRecords);
 
     // Write 2 (updates)
@@ -335,7 +336,7 @@
     numRecords = 50;
 
     deleteBatch(hoodieWriteConfig, client, newCommitTime, prevCommitTime,
-        initCommitTime, numRecords, HoodieWriteClient::delete, isPrepped, true,
+        initCommitTime, numRecords, SparkRDDWriteClient::delete, isPrepped, true,
         0, 150);
 
     // Now simulate an upgrade and perform a restore operation
@@ -358,7 +359,7 @@
     numRecords = 50;
 
     deleteBatch(newConfig, client, newCommitTime, prevCommitTime,
-        initCommitTime, numRecords, HoodieWriteClient::delete, isPrepped, true,
+        initCommitTime, numRecords, SparkRDDWriteClient::delete, isPrepped, true,
         0, 150);
 
     HoodieActiveTimeline activeTimeline = new HoodieActiveTimeline(metaClient, false);
@@ -382,7 +383,7 @@
    */
   @Test
   public void testDeletes() throws Exception {
-    HoodieWriteClient client = getHoodieWriteClient(getConfig(), false);
+    SparkRDDWriteClient client = getHoodieWriteClient(getConfig(), false);
 
     /**
      * Write 1 (inserts and deletes) Write actual 200 insert records and ignore 100 delete records
@@ -402,7 +403,7 @@
         };
     writeBatch(client, newCommitTime, initCommitTime, Option.empty(), initCommitTime,
         // unused as genFn uses hard-coded number of inserts/updates/deletes
-        -1, recordGenFunction, HoodieWriteClient::upsert, true, 200, 200, 1);
+        -1, recordGenFunction, SparkRDDWriteClient::upsert, true, 200, 200, 1);
 
     /**
      * Write 2 (deletes+writes).
@@ -419,7 +420,7 @@
       return recordsInSecondBatch;
     };
     writeBatch(client, newCommitTime, prevCommitTime, Option.empty(), initCommitTime, 100, recordGenFunction,
-        HoodieWriteClient::upsert, true, 50, 150, 2);
+        SparkRDDWriteClient::upsert, true, 50, 150, 2);
   }
 
   /**
@@ -429,7 +430,7 @@
    */
   @Test
   public void testDeletesForInsertsInSameBatch() throws Exception {
-    HoodieWriteClient client = getHoodieWriteClient(getConfig(), false);
+    SparkRDDWriteClient client = getHoodieWriteClient(getConfig(), false);
 
     /**
      * Write 200 inserts and issue deletes to a subset(50) of inserts.
@@ -449,7 +450,7 @@
         };
 
     writeBatch(client, newCommitTime, initCommitTime, Option.empty(), initCommitTime,
-        -1, recordGenFunction, HoodieWriteClient::upsert, true, 150, 150, 1);
+        -1, recordGenFunction, SparkRDDWriteClient::upsert, true, 150, 150, 1);
   }
 
   /**
@@ -458,7 +459,7 @@
   @ParameterizedTest
   @EnumSource(value = IndexType.class, names = {"GLOBAL_BLOOM", "GLOBAL_SIMPLE"})
   public void testUpsertsUpdatePartitionPathGlobalBloom(IndexType indexType) throws Exception {
-    testUpsertsUpdatePartitionPath(indexType, getConfig(), HoodieWriteClient::upsert);
+    testUpsertsUpdatePartitionPath(indexType, getConfig(), SparkRDDWriteClient::upsert);
   }
 
   /**
@@ -476,7 +477,7 @@
    * @param writeFn write function to be used for testing
    */
   private void testUpsertsUpdatePartitionPath(IndexType indexType, HoodieWriteConfig config,
-      Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieRecord>, String> writeFn)
+      Function3<JavaRDD<WriteStatus>, SparkRDDWriteClient, JavaRDD<HoodieRecord>, String> writeFn)
       throws Exception {
     // instantiate client
 
@@ -491,7 +492,7 @@
     HoodieTableMetaClient.initTableType(metaClient.getHadoopConf(), metaClient.getBasePath(),
         metaClient.getTableType(), metaClient.getTableConfig().getTableName(), metaClient.getArchivePath(),
         metaClient.getTableConfig().getPayloadClass(), VERSION_0);
-    HoodieWriteClient client = getHoodieWriteClient(hoodieWriteConfig, false);
+    SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig, false);
 
     // Write 1
     String newCommitTime = "001";
@@ -637,7 +638,7 @@
     HoodieWriteConfig config = getSmallInsertWriteConfig(insertSplitLimit); // hold upto 200 records max
     dataGen = new HoodieTestDataGenerator(new String[] {testPartitionPath});
 
-    HoodieWriteClient client = getHoodieWriteClient(config, false);
+    SparkRDDWriteClient client = getHoodieWriteClient(config, false);
 
     // Inserts => will write file1
     String commitTime1 = "001";
@@ -748,7 +749,7 @@
     // setup the small file handling params
     HoodieWriteConfig config = getSmallInsertWriteConfig(insertSplitLimit); // hold upto 200 records max
     dataGen = new HoodieTestDataGenerator(new String[] {testPartitionPath});
-    HoodieWriteClient client = getHoodieWriteClient(config, false);
+    SparkRDDWriteClient client = getHoodieWriteClient(config, false);
 
     // Inserts => will write file1
     String commitTime1 = "001";
@@ -829,7 +830,7 @@
     HoodieWriteConfig config = getSmallInsertWriteConfig(insertSplitLimit); // hold upto 200 records max
     dataGen = new HoodieTestDataGenerator(new String[] {testPartitionPath});
 
-    HoodieWriteClient client = getHoodieWriteClient(config, false);
+    SparkRDDWriteClient client = getHoodieWriteClient(config, false);
 
     // Inserts => will write file1
     String commitTime1 = "001";
@@ -920,7 +921,7 @@
   private void verifyInsertOverwritePartitionHandling(int batch1RecordsCount, int batch2RecordsCount) throws Exception {
     final String testPartitionPath = "americas";
     HoodieWriteConfig config = getSmallInsertWriteConfig(2000, false);
-    HoodieWriteClient client = getHoodieWriteClient(config, false);
+    SparkRDDWriteClient client = getHoodieWriteClient(config, false);
     dataGen = new HoodieTestDataGenerator(new String[] {testPartitionPath});
 
     // Do Inserts
@@ -968,7 +969,7 @@
     }
   }
 
-  private Pair<Set<String>, List<HoodieRecord>> testUpdates(String instantTime, HoodieWriteClient client,
+  private Pair<Set<String>, List<HoodieRecord>> testUpdates(String instantTime, SparkRDDWriteClient client,
       int sizeToInsertAndUpdate, int expectedTotalRecords)
       throws IOException {
     client.startCommitWithTime(instantTime);
@@ -993,7 +994,7 @@
     return Pair.of(keys, inserts);
   }
 
-  private void testDeletes(HoodieWriteClient client, List<HoodieRecord> previousRecords, int sizeToDelete,
+  private void testDeletes(SparkRDDWriteClient client, List<HoodieRecord> previousRecords, int sizeToDelete,
       String existingFile, String instantTime, int exepctedRecords, List<String> keys) {
     client.startCommitWithTime(instantTime);
 
@@ -1039,7 +1040,7 @@
     HoodieWriteConfig config = getSmallInsertWriteConfig(insertSplitLimit, true); // hold upto 200 records max
     dataGen = new HoodieTestDataGenerator(new String[] {testPartitionPath});
 
-    HoodieWriteClient client = getHoodieWriteClient(config, false);
+    SparkRDDWriteClient client = getHoodieWriteClient(config, false);
 
     // delete non existent keys
     String commitTime1 = "001";
@@ -1060,9 +1061,9 @@
   public void testCommitWritesRelativePaths() throws Exception {
 
     HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build();
-    try (HoodieWriteClient client = getHoodieWriteClient(cfg);) {
+    try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) {
       HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf, basePath);
-      HoodieTable table = HoodieTable.create(metaClient, cfg, hadoopConf);
+      HoodieSparkTable table = HoodieSparkTable.create(cfg, context, metaClient);
 
       String instantTime = "000";
       client.startCommitWithTime(instantTime);
@@ -1104,7 +1105,7 @@
   @Test
   public void testMetadataStatsOnCommit() throws Exception {
     HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build();
-    HoodieWriteClient client = getHoodieWriteClient(cfg);
+    SparkRDDWriteClient client = getHoodieWriteClient(cfg);
 
     String instantTime0 = "000";
     client.startCommitWithTime(instantTime0);
@@ -1170,7 +1171,7 @@
     String instantTime = "000";
     HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder()
         .withEnableOptimisticConsistencyGuard(enableOptimisticConsistencyGuard).build()).build();
-    HoodieWriteClient client = getHoodieWriteClient(cfg);
+    SparkRDDWriteClient client = getHoodieWriteClient(cfg);
     Pair<Path, JavaRDD<WriteStatus>> result = testConsistencyCheck(metaClient, instantTime, enableOptimisticConsistencyGuard);
 
     // Delete orphan marker and commit should succeed
@@ -1200,7 +1201,7 @@
             .withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder()
                 .withConsistencyCheckEnabled(true)
                 .withOptimisticConsistencyGuardSleepTimeMs(1).build()).build();
-    HoodieWriteClient client = getHoodieWriteClient(cfg);
+    SparkRDDWriteClient client = getHoodieWriteClient(cfg);
     testConsistencyCheck(metaClient, instantTime, enableOptimisticConsistencyGuard);
 
     if (!enableOptimisticConsistencyGuard) {
@@ -1254,7 +1255,7 @@
         .withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true)
             .withOptimisticConsistencyGuardSleepTimeMs(1).build())
         .build());
-    HoodieWriteClient client = getHoodieWriteClient(cfg);
+    SparkRDDWriteClient client = getHoodieWriteClient(cfg);
 
     client.startCommitWithTime(instantTime);
     JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(dataGen.generateInserts(instantTime, 200), 1);
@@ -1291,7 +1292,7 @@
     HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false)
         .withAllowMultiWriteOnSameInstant(true)
         .build();
-    HoodieWriteClient client = getHoodieWriteClient(cfg);
+    SparkRDDWriteClient client = getHoodieWriteClient(cfg);
     String firstInstantTime = "0000";
     client.startCommitWithTime(firstInstantTime);
     int numRecords = 200;
diff --git a/hudi-client/src/test/java/org/apache/hudi/client/TestHoodieInternalWriteStatus.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieInternalWriteStatus.java
similarity index 100%
rename from hudi-client/src/test/java/org/apache/hudi/client/TestHoodieInternalWriteStatus.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieInternalWriteStatus.java
diff --git a/hudi-client/src/test/java/org/apache/hudi/client/TestHoodieReadClient.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieReadClient.java
similarity index 87%
rename from hudi-client/src/test/java/org/apache/hudi/client/TestHoodieReadClient.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieReadClient.java
index 80af513..35ee557 100644
--- a/hudi-client/src/test/java/org/apache/hudi/client/TestHoodieReadClient.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieReadClient.java
@@ -52,7 +52,7 @@
    */
   @Test
   public void testReadFilterExistAfterInsert() throws Exception {
-    testReadFilterExist(getConfig(), HoodieWriteClient::insert);
+    testReadFilterExist(getConfig(), SparkRDDWriteClient::insert);
   }
 
   /**
@@ -60,7 +60,7 @@
    */
   @Test
   public void testReadFilterExistAfterInsertPrepped() throws Exception {
-    testReadFilterExist(getConfig(), HoodieWriteClient::insertPreppedRecords);
+    testReadFilterExist(getConfig(), SparkRDDWriteClient::insertPreppedRecords);
   }
 
   /**
@@ -68,7 +68,7 @@
    */
   @Test
   public void testReadFilterExistAfterBulkInsert() throws Exception {
-    testReadFilterExist(getConfigBuilder().withBulkInsertParallelism(1).build(), HoodieWriteClient::bulkInsert);
+    testReadFilterExist(getConfigBuilder().withBulkInsertParallelism(1).build(), SparkRDDWriteClient::bulkInsert);
   }
 
   /**
@@ -84,7 +84,7 @@
 
   @Test
   public void testReadROViewFailsWithoutSqlContext() {
-    HoodieReadClient readClient = new HoodieReadClient(jsc, getConfig());
+    HoodieReadClient readClient = new HoodieReadClient(context, getConfig());
     JavaRDD<HoodieKey> recordsRDD = jsc.parallelize(new ArrayList<>(), 1);
     assertThrows(IllegalStateException.class, () -> {
       readClient.readROView(recordsRDD, 1);
@@ -100,8 +100,8 @@
    * @throws Exception in case of error
    */
   private void testReadFilterExist(HoodieWriteConfig config,
-      Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieRecord>, String> writeFn) throws Exception {
-    try (HoodieWriteClient writeClient = getHoodieWriteClient(config);) {
+      Function3<JavaRDD<WriteStatus>, SparkRDDWriteClient, JavaRDD<HoodieRecord>, String> writeFn) throws Exception {
+    try (SparkRDDWriteClient writeClient = getHoodieWriteClient(config);) {
       HoodieReadClient readClient = getHoodieReadClient(config.getBasePath());
       String newCommitTime = writeClient.startCommit();
       List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
@@ -126,7 +126,7 @@
 
       // check path exists for written keys
       JavaPairRDD<HoodieKey, Option<String>> keyToPathPair =
-              anotherReadClient.checkExists(recordsRDD.map(r -> r.getKey()));
+              anotherReadClient.checkExists(recordsRDD.map(HoodieRecord::getKey));
       JavaRDD<HoodieKey> keysWithPaths = keyToPathPair.filter(keyPath -> keyPath._2.isPresent())
               .map(keyPath -> keyPath._1);
       assertEquals(75, keysWithPaths.count());
@@ -153,7 +153,7 @@
    */
   @Test
   public void testTagLocationAfterInsert() throws Exception {
-    testTagLocation(getConfig(), HoodieWriteClient::insert, HoodieWriteClient::upsert, false);
+    testTagLocation(getConfig(), SparkRDDWriteClient::insert, SparkRDDWriteClient::upsert, false);
   }
 
   /**
@@ -161,7 +161,7 @@
    */
   @Test
   public void testTagLocationAfterInsertPrepped() throws Exception {
-    testTagLocation(getConfig(), HoodieWriteClient::insertPreppedRecords, HoodieWriteClient::upsertPreppedRecords,
+    testTagLocation(getConfig(), SparkRDDWriteClient::insertPreppedRecords, SparkRDDWriteClient::upsertPreppedRecords,
         true);
   }
 
@@ -170,8 +170,8 @@
    */
   @Test
   public void testTagLocationAfterBulkInsert() throws Exception {
-    testTagLocation(getConfigBuilder().withBulkInsertParallelism(1).build(), HoodieWriteClient::bulkInsert,
-        HoodieWriteClient::upsert, false);
+    testTagLocation(getConfigBuilder().withBulkInsertParallelism(1).build(), SparkRDDWriteClient::bulkInsert,
+        SparkRDDWriteClient::upsert, false);
   }
 
   /**
@@ -182,7 +182,7 @@
     testTagLocation(
         getConfigBuilder().withBulkInsertParallelism(1).build(), (writeClient, recordRDD, instantTime) -> writeClient
             .bulkInsertPreppedRecords(recordRDD, instantTime, Option.empty()),
-        HoodieWriteClient::upsertPreppedRecords, true);
+        SparkRDDWriteClient::upsertPreppedRecords, true);
   }
 
   /**
@@ -195,10 +195,10 @@
    * @throws Exception in case of error
    */
   private void testTagLocation(HoodieWriteConfig hoodieWriteConfig,
-      Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieRecord>, String> insertFn,
-      Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieRecord>, String> updateFn, boolean isPrepped)
+      Function3<JavaRDD<WriteStatus>, SparkRDDWriteClient, JavaRDD<HoodieRecord>, String> insertFn,
+      Function3<JavaRDD<WriteStatus>, SparkRDDWriteClient, JavaRDD<HoodieRecord>, String> updateFn, boolean isPrepped)
       throws Exception {
-    try (HoodieWriteClient client = getHoodieWriteClient(hoodieWriteConfig);) {
+    try (SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig);) {
       // Write 1 (only inserts)
       String newCommitTime = "001";
       String initCommitTime = "000";
diff --git a/hudi-client/src/test/java/org/apache/hudi/client/TestMultiFS.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestMultiFS.java
similarity index 96%
rename from hudi-client/src/test/java/org/apache/hudi/client/TestMultiFS.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestMultiFS.java
index b4369d8..34daed7 100644
--- a/hudi-client/src/test/java/org/apache/hudi/client/TestMultiFS.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestMultiFS.java
@@ -82,8 +82,8 @@
     HoodieWriteConfig cfg = getHoodieWriteConfig(dfsBasePath);
     HoodieWriteConfig localConfig = getHoodieWriteConfig(tablePath);
 
-    try (HoodieWriteClient hdfsWriteClient = getHoodieWriteClient(cfg);
-        HoodieWriteClient localWriteClient = getHoodieWriteClient(localConfig)) {
+    try (SparkRDDWriteClient hdfsWriteClient = getHoodieWriteClient(cfg);
+         SparkRDDWriteClient localWriteClient = getHoodieWriteClient(localConfig)) {
 
       // Write generated data to hdfs (only inserts)
       String readCommitTime = hdfsWriteClient.startCommit();
diff --git a/hudi-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java
similarity index 91%
rename from hudi-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java
index 0e48746..8ee0c16 100644
--- a/hudi-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java
@@ -132,12 +132,12 @@
         metaClient.getArchivePath(), metaClient.getTableConfig().getPayloadClass(), VERSION_1);
 
     HoodieWriteConfig hoodieWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA);
-    HoodieWriteClient client = getHoodieWriteClient(hoodieWriteConfig, false);
+    SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig, false);
 
     // Initial inserts with TRIP_EXAMPLE_SCHEMA
     int numRecords = 10;
     insertFirstBatch(hoodieWriteConfig, client, "001", initCommitTime,
-                     numRecords, HoodieWriteClient::insert, false, false, numRecords);
+                     numRecords, SparkRDDWriteClient::insert, false, false, numRecords);
     checkLatestDeltaCommit("001");
 
     // Compact once so we can incrementally read later
@@ -147,7 +147,7 @@
     // Updates with same schema is allowed
     final int numUpdateRecords = 5;
     updateBatch(hoodieWriteConfig, client, "003", "002", Option.empty(),
-                initCommitTime, numUpdateRecords, HoodieWriteClient::upsert, false, false, 0, 0, 0);
+                initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, false, 0, 0, 0);
     checkLatestDeltaCommit("003");
     checkReadRecords("000", numRecords);
 
@@ -155,7 +155,7 @@
     final int numDeleteRecords = 2;
     numRecords -= numDeleteRecords;
     deleteBatch(hoodieWriteConfig, client, "004", "003", initCommitTime, numDeleteRecords,
-                HoodieWriteClient::delete, false, false, 0, 0);
+        SparkRDDWriteClient::delete, false, false, 0, 0);
     checkLatestDeltaCommit("004");
     checkReadRecords("000", numRecords);
 
@@ -167,7 +167,7 @@
       // We cannot use insertBatch directly here because we want to insert records
       // with a devolved schema and insertBatch inserts records using the TRIP_EXMPLE_SCHEMA.
       writeBatch(client, "005", "004", Option.empty(), "003", numRecords,
-          (String s, Integer a) -> failedRecords, HoodieWriteClient::insert, false, 0, 0, 0);
+          (String s, Integer a) -> failedRecords, SparkRDDWriteClient::insert, false, 0, 0, 0);
       fail("Insert with devolved scheme should fail");
     } catch (HoodieInsertException ex) {
       // no new commit
@@ -179,7 +179,7 @@
     // Update with devolved schema is also not allowed
     try {
       updateBatch(hoodieDevolvedWriteConfig, client, "005", "004", Option.empty(),
-                  initCommitTime, numUpdateRecords, HoodieWriteClient::upsert, false, false, 0, 0, 0);
+                  initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, false, 0, 0, 0);
       fail("Update with devolved scheme should fail");
     } catch (HoodieUpsertException ex) {
       // no new commit
@@ -196,7 +196,7 @@
     // with a evolved schemaand insertBatch inserts records using the TRIP_EXMPLE_SCHEMA.
     final List<HoodieRecord> evolvedRecords = generateInsertsWithSchema("005", numRecords, TRIP_EXAMPLE_SCHEMA_EVOLVED);
     writeBatch(client, "005", "004", Option.empty(), initCommitTime, numRecords,
-        (String s, Integer a) -> evolvedRecords, HoodieWriteClient::insert, false, 0, 0, 0);
+        (String s, Integer a) -> evolvedRecords, SparkRDDWriteClient::insert, false, 0, 0, 0);
 
     // new commit
     checkLatestDeltaCommit("005");
@@ -205,7 +205,7 @@
     // Updates with evolved schema is allowed
     final List<HoodieRecord> updateRecords = generateUpdatesWithSchema("006", numUpdateRecords, TRIP_EXAMPLE_SCHEMA_EVOLVED);
     writeBatch(client, "006", "005", Option.empty(), initCommitTime,
-        numUpdateRecords, (String s, Integer a) -> updateRecords, HoodieWriteClient::upsert, false, 0, 0, 0);
+        numUpdateRecords, (String s, Integer a) -> updateRecords, SparkRDDWriteClient::upsert, false, 0, 0, 0);
     // new commit
     checkLatestDeltaCommit("006");
     checkReadRecords("000", 2 * numRecords);
@@ -215,7 +215,7 @@
     client = getHoodieWriteClient(hoodieWriteConfig, false);
     try {
       updateBatch(hoodieWriteConfig, client, "007", "006", Option.empty(),
-                  initCommitTime, numUpdateRecords, HoodieWriteClient::upsert, false, false, 0, 0, 0);
+                  initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, false, 0, 0, 0);
       fail("Update with original scheme should fail");
     } catch (HoodieUpsertException ex) {
       // no new commit
@@ -233,7 +233,7 @@
       failedRecords.clear();
       failedRecords.addAll(dataGen.generateInserts("007", numRecords));
       writeBatch(client, "007", "006", Option.empty(), initCommitTime, numRecords,
-          (String s, Integer a) -> failedRecords, HoodieWriteClient::insert, true, numRecords, numRecords, 1);
+          (String s, Integer a) -> failedRecords, SparkRDDWriteClient::insert, true, numRecords, numRecords, 1);
       fail("Insert with original scheme should fail");
     } catch (HoodieInsertException ex) {
       // no new commit
@@ -257,13 +257,13 @@
     // Updates with original schema are now allowed
     client = getHoodieWriteClient(hoodieWriteConfig, false);
     updateBatch(hoodieWriteConfig, client, "008", "004", Option.empty(),
-                initCommitTime, numUpdateRecords, HoodieWriteClient::upsert, false, false, 0, 0, 0);
+                initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, false, 0, 0, 0);
     // new commit
     checkLatestDeltaCommit("008");
     checkReadRecords("000", 2 * numRecords);
 
     // Insert with original schema is allowed now
-    insertBatch(hoodieWriteConfig, client, "009", "008", numRecords, HoodieWriteClient::insert,
+    insertBatch(hoodieWriteConfig, client, "009", "008", numRecords, SparkRDDWriteClient::insert,
         false, false, 0, 0, 0);
     checkLatestDeltaCommit("009");
     checkReadRecords("000", 3 * numRecords);
@@ -277,18 +277,18 @@
         metaClient.getArchivePath(), metaClient.getTableConfig().getPayloadClass(), VERSION_1);
 
     HoodieWriteConfig hoodieWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA);
-    HoodieWriteClient client = getHoodieWriteClient(hoodieWriteConfig, false);
+    SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig, false);
 
     // Initial inserts with TRIP_EXAMPLE_SCHEMA
     int numRecords = 10;
     insertFirstBatch(hoodieWriteConfig, client, "001", initCommitTime,
-                     numRecords, HoodieWriteClient::insert, false, true, numRecords);
+                     numRecords, SparkRDDWriteClient::insert, false, true, numRecords);
     checkReadRecords("000", numRecords);
 
     // Updates with same schema is allowed
     final int numUpdateRecords = 5;
     updateBatch(hoodieWriteConfig, client, "002", "001", Option.empty(),
-                initCommitTime, numUpdateRecords, HoodieWriteClient::upsert, false, true,
+                initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, true,
                 numUpdateRecords, numRecords, 2);
     checkReadRecords("000", numRecords);
 
@@ -296,7 +296,7 @@
     final int numDeleteRecords = 2;
     numRecords -= numDeleteRecords;
     deleteBatch(hoodieWriteConfig, client, "003", "002", initCommitTime, numDeleteRecords,
-                HoodieWriteClient::delete, false, true, 0, numRecords);
+        SparkRDDWriteClient::delete, false, true, 0, numRecords);
     checkReadRecords("000", numRecords);
 
     // Insert with devolved schema is not allowed
@@ -307,7 +307,7 @@
       // We cannot use insertBatch directly here because we want to insert records
       // with a devolved schema.
       writeBatch(client, "004", "003", Option.empty(), "003", numRecords,
-          (String s, Integer a) -> failedRecords, HoodieWriteClient::insert, true, numRecords, numRecords, 1);
+          (String s, Integer a) -> failedRecords, SparkRDDWriteClient::insert, true, numRecords, numRecords, 1);
       fail("Insert with devolved scheme should fail");
     } catch (HoodieInsertException ex) {
       // no new commit
@@ -319,7 +319,7 @@
     // Update with devolved schema is not allowed
     try {
       updateBatch(hoodieDevolvedWriteConfig, client, "004", "003", Option.empty(),
-                  initCommitTime, numUpdateRecords, HoodieWriteClient::upsert, false, true,
+                  initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, true,
                   numUpdateRecords, 2 * numRecords, 5);
       fail("Update with devolved scheme should fail");
     } catch (HoodieUpsertException ex) {
@@ -336,7 +336,7 @@
     // We cannot use insertBatch directly here because we want to insert records
     // with a evolved schema.
     writeBatch(client, "004", "003", Option.empty(), initCommitTime, numRecords,
-        (String s, Integer a) -> evolvedRecords, HoodieWriteClient::insert, true, numRecords, 2 * numRecords, 4);
+        (String s, Integer a) -> evolvedRecords, SparkRDDWriteClient::insert, true, numRecords, 2 * numRecords, 4);
     // new commit
     HoodieTimeline curTimeline = metaClient.reloadActiveTimeline().getCommitTimeline().filterCompletedInstants();
     assertTrue(curTimeline.lastInstant().get().getTimestamp().equals("004"));
@@ -345,7 +345,7 @@
     // Updates with evolved schema is allowed
     final List<HoodieRecord> updateRecords = generateUpdatesWithSchema("005", numUpdateRecords, TRIP_EXAMPLE_SCHEMA_EVOLVED);
     writeBatch(client, "005", "004", Option.empty(), initCommitTime,
-        numUpdateRecords, (String s, Integer a) -> updateRecords, HoodieWriteClient::upsert, true, numUpdateRecords, 2 * numRecords, 5);
+        numUpdateRecords, (String s, Integer a) -> updateRecords, SparkRDDWriteClient::upsert, true, numUpdateRecords, 2 * numRecords, 5);
     checkReadRecords("000", 2 * numRecords);
 
     // Now even the original schema cannot be used for updates as it is devolved
@@ -353,7 +353,7 @@
     client = getHoodieWriteClient(hoodieWriteConfig, false);
     try {
       updateBatch(hoodieWriteConfig, client, "006", "005", Option.empty(),
-                  initCommitTime, numUpdateRecords, HoodieWriteClient::upsert, false, true,
+                  initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, true,
                   numUpdateRecords, numRecords, 2);
       fail("Update with original scheme should fail");
     } catch (HoodieUpsertException ex) {
@@ -372,7 +372,7 @@
       failedRecords.clear();
       failedRecords.addAll(dataGen.generateInserts("006", numRecords));
       writeBatch(client, "006", "005", Option.empty(), initCommitTime, numRecords,
-          (String s, Integer a) -> failedRecords, HoodieWriteClient::insert, true, numRecords, numRecords, 1);
+          (String s, Integer a) -> failedRecords, SparkRDDWriteClient::insert, true, numRecords, numRecords, 1);
       fail("Insert with original scheme should fail");
     } catch (HoodieInsertException ex) {
       // no new commit
@@ -397,13 +397,13 @@
     checkReadRecords("000", numRecords);
 
     // Insert with original schema is allowed now
-    insertBatch(hoodieWriteConfig, client, "007", "003", numRecords, HoodieWriteClient::insert,
+    insertBatch(hoodieWriteConfig, client, "007", "003", numRecords, SparkRDDWriteClient::insert,
         false, true, numRecords, 2 * numRecords, 1);
     checkReadRecords("000", 2 * numRecords);
 
     // Update with original schema is allowed now
     updateBatch(hoodieWriteConfig, client, "008", "007", Option.empty(),
-        initCommitTime, numUpdateRecords, HoodieWriteClient::upsert, false, true,
+        initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, true,
         numUpdateRecords, 2 * numRecords, 5);
     checkReadRecords("000", 2 * numRecords);
   }
diff --git a/hudi-client/src/test/java/org/apache/hudi/client/TestUpdateSchemaEvolution.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestUpdateSchemaEvolution.java
similarity index 96%
rename from hudi-client/src/test/java/org/apache/hudi/client/TestUpdateSchemaEvolution.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestUpdateSchemaEvolution.java
index 8b42671..8058304 100644
--- a/hudi-client/src/test/java/org/apache/hudi/client/TestUpdateSchemaEvolution.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestUpdateSchemaEvolution.java
@@ -29,7 +29,7 @@
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.io.HoodieCreateHandle;
 import org.apache.hudi.io.HoodieMergeHandle;
-import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.table.HoodieSparkTable;
 import org.apache.hudi.testutils.HoodieClientTestHarness;
 
 import org.apache.avro.Schema;
@@ -72,7 +72,7 @@
   public void testSchemaEvolutionOnUpdate() throws Exception {
     // Create a bunch of records with a old version of schema
     final HoodieWriteConfig config = makeHoodieClientConfig("/exampleSchema.txt");
-    final HoodieTable<?> table = HoodieTable.create(config, hadoopConf);
+    final HoodieSparkTable table = HoodieSparkTable.create(config, context);
 
     final List<WriteStatus> statuses = jsc.parallelize(Arrays.asList(1)).map(x -> {
       String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\","
@@ -109,7 +109,7 @@
     final WriteStatus insertResult = statuses.get(0);
     String fileId = insertResult.getFileId();
 
-    final HoodieTable table2 = HoodieTable.create(config2, hadoopConf);
+    final HoodieSparkTable table2 = HoodieSparkTable.create(config, context);
     assertEquals(1, jsc.parallelize(Arrays.asList(1)).map(x -> {
       // New content with values for the newly added field
       String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\","
diff --git a/hudi-client/src/test/java/org/apache/hudi/client/TestWriteStatus.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestWriteStatus.java
similarity index 100%
rename from hudi-client/src/test/java/org/apache/hudi/client/TestWriteStatus.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestWriteStatus.java
diff --git a/hudi-client/src/test/java/org/apache/hudi/client/bootstrap/TestBootstrapRegexModeSelector.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/bootstrap/TestBootstrapRegexModeSelector.java
similarity index 100%
rename from hudi-client/src/test/java/org/apache/hudi/client/bootstrap/TestBootstrapRegexModeSelector.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/bootstrap/TestBootstrapRegexModeSelector.java
diff --git a/hudi-client/src/test/java/org/apache/hudi/client/bootstrap/TestUniformBootstrapModeSelector.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/bootstrap/TestUniformBootstrapModeSelector.java
similarity index 100%
rename from hudi-client/src/test/java/org/apache/hudi/client/bootstrap/TestUniformBootstrapModeSelector.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/bootstrap/TestUniformBootstrapModeSelector.java
diff --git a/hudi-client/src/test/java/org/apache/hudi/client/model/TestHoodieInternalRow.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/model/TestHoodieInternalRow.java
similarity index 100%
rename from hudi-client/src/test/java/org/apache/hudi/client/model/TestHoodieInternalRow.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/model/TestHoodieInternalRow.java
diff --git a/hudi-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryQueue.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryQueue.java
similarity index 92%
rename from hudi-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryQueue.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryQueue.java
index 5520ba8..c30635b 100644
--- a/hudi-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryQueue.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestBoundedInMemoryQueue.java
@@ -30,7 +30,6 @@
 import org.apache.hudi.common.util.queue.FunctionBasedQueueProducer;
 import org.apache.hudi.common.util.queue.IteratorBasedQueueProducer;
 import org.apache.hudi.exception.HoodieException;
-import org.apache.hudi.execution.LazyInsertIterable.HoodieInsertValueGenResult;
 import org.apache.hudi.testutils.HoodieClientTestHarness;
 
 import org.apache.avro.generic.IndexedRecord;
@@ -53,7 +52,7 @@
 
 import scala.Tuple2;
 
-import static org.apache.hudi.execution.LazyInsertIterable.getTransformFunction;
+import static org.apache.hudi.execution.HoodieLazyInsertIterable.getTransformFunction;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertThrows;
@@ -83,7 +82,7 @@
   public void testRecordReading() throws Exception {
     final int numRecords = 128;
     final List<HoodieRecord> hoodieRecords = dataGen.generateInserts(instantTime, numRecords);
-    final BoundedInMemoryQueue<HoodieRecord, HoodieInsertValueGenResult<HoodieRecord>> queue =
+    final BoundedInMemoryQueue<HoodieRecord, HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord>> queue =
         new BoundedInMemoryQueue(FileIOUtils.KB, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA));
     // Produce
     Future<Boolean> resFuture = executorService.submit(() -> {
@@ -97,7 +96,7 @@
       final HoodieRecord originalRecord = originalRecordIterator.next();
       final Option<IndexedRecord> originalInsertValue =
           originalRecord.getData().getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA);
-      final HoodieInsertValueGenResult<HoodieRecord> payload = queue.iterator().next();
+      final HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord> payload = queue.iterator().next();
       // Ensure that record ordering is guaranteed.
       assertEquals(originalRecord, payload.record);
       // cached insert value matches the expected insert value.
@@ -123,7 +122,7 @@
     final int numProducers = 40;
     final List<List<HoodieRecord>> recs = new ArrayList<>();
 
-    final BoundedInMemoryQueue<HoodieRecord, HoodieInsertValueGenResult<HoodieRecord>> queue =
+    final BoundedInMemoryQueue<HoodieRecord, HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord>> queue =
         new BoundedInMemoryQueue(FileIOUtils.KB, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA));
 
     // Record Key to <Producer Index, Rec Index within a producer>
@@ -189,7 +188,7 @@
 
     // Read recs and ensure we have covered all producer recs.
     while (queue.iterator().hasNext()) {
-      final HoodieInsertValueGenResult<HoodieRecord> payload = queue.iterator().next();
+      final HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord> payload = queue.iterator().next();
       final HoodieRecord rec = payload.record;
       Tuple2<Integer, Integer> producerPos = keyToProducerAndIndexMap.get(rec.getRecordKey());
       Integer lastSeenPos = lastSeenMap.get(producerPos._1());
@@ -217,12 +216,12 @@
     final List<HoodieRecord> hoodieRecords = dataGen.generateInserts(instantTime, numRecords);
     // maximum number of records to keep in memory.
     final int recordLimit = 5;
-    final SizeEstimator<HoodieInsertValueGenResult<HoodieRecord>> sizeEstimator = new DefaultSizeEstimator<>();
-    HoodieInsertValueGenResult<HoodieRecord> payload =
+    final SizeEstimator<HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord>> sizeEstimator = new DefaultSizeEstimator<>();
+    HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord> payload =
         getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA).apply(hoodieRecords.get(0));
     final long objSize = sizeEstimator.sizeEstimate(payload);
     final long memoryLimitInBytes = recordLimit * objSize;
-    final BoundedInMemoryQueue<HoodieRecord, HoodieInsertValueGenResult<HoodieRecord>> queue =
+    final BoundedInMemoryQueue<HoodieRecord, HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord>> queue =
         new BoundedInMemoryQueue(memoryLimitInBytes, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA));
 
     // Produce
@@ -267,7 +266,7 @@
     final List<HoodieRecord> hoodieRecords = dataGen.generateInserts(instantTime, numRecords);
     final SizeEstimator<Tuple2<HoodieRecord, Option<IndexedRecord>>> sizeEstimator = new DefaultSizeEstimator<>();
     // queue memory limit
-    HoodieInsertValueGenResult<HoodieRecord> payload =
+    HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord> payload =
         getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA).apply(hoodieRecords.get(0));
     final long objSize = sizeEstimator.sizeEstimate(new Tuple2<>(payload.record, payload.insertValue));
     final long memoryLimitInBytes = 4 * objSize;
diff --git a/hudi-client/src/test/java/org/apache/hudi/execution/TestSparkBoundedInMemoryExecutor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestSparkBoundedInMemoryExecutor.java
similarity index 87%
rename from hudi-client/src/test/java/org/apache/hudi/execution/TestSparkBoundedInMemoryExecutor.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestSparkBoundedInMemoryExecutor.java
index 487dd22..fd41a16 100644
--- a/hudi-client/src/test/java/org/apache/hudi/execution/TestSparkBoundedInMemoryExecutor.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/TestSparkBoundedInMemoryExecutor.java
@@ -24,7 +24,6 @@
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer;
 import org.apache.hudi.config.HoodieWriteConfig;
-import org.apache.hudi.execution.LazyInsertIterable.HoodieInsertValueGenResult;
 import org.apache.hudi.testutils.HoodieClientTestHarness;
 
 import org.apache.avro.generic.IndexedRecord;
@@ -36,7 +35,7 @@
 
 import scala.Tuple2;
 
-import static org.apache.hudi.execution.LazyInsertIterable.getTransformFunction;
+import static org.apache.hudi.execution.HoodieLazyInsertIterable.getTransformFunction;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.mockito.Mockito.mock;
@@ -63,13 +62,13 @@
 
     HoodieWriteConfig hoodieWriteConfig = mock(HoodieWriteConfig.class);
     when(hoodieWriteConfig.getWriteBufferLimitBytes()).thenReturn(1024);
-    BoundedInMemoryQueueConsumer<HoodieInsertValueGenResult<HoodieRecord>, Integer> consumer =
-        new BoundedInMemoryQueueConsumer<HoodieInsertValueGenResult<HoodieRecord>, Integer>() {
+    BoundedInMemoryQueueConsumer<HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord>, Integer> consumer =
+        new BoundedInMemoryQueueConsumer<HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord>, Integer>() {
 
           private int count = 0;
 
           @Override
-          protected void consumeOneRecord(HoodieInsertValueGenResult<HoodieRecord> record) {
+          protected void consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord> record) {
             count++;
           }
 
diff --git a/hudi-client/src/test/java/org/apache/hudi/execution/bulkinsert/TestBulkInsertInternalPartitioner.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/bulkinsert/TestBulkInsertInternalPartitioner.java
similarity index 97%
rename from hudi-client/src/test/java/org/apache/hudi/execution/bulkinsert/TestBulkInsertInternalPartitioner.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/bulkinsert/TestBulkInsertInternalPartitioner.java
index b8ebd5a..834229b 100644
--- a/hudi-client/src/test/java/org/apache/hudi/execution/bulkinsert/TestBulkInsertInternalPartitioner.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/execution/bulkinsert/TestBulkInsertInternalPartitioner.java
@@ -80,7 +80,7 @@
                                                  boolean isGloballySorted, boolean isLocallySorted,
                                                  Map<String, Long> expectedPartitionNumRecords) {
     int numPartitions = 2;
-    JavaRDD<HoodieRecord> actualRecords = partitioner.repartitionRecords(records, numPartitions);
+    JavaRDD<HoodieRecord> actualRecords = (JavaRDD<HoodieRecord>) partitioner.repartitionRecords(records, numPartitions);
     assertEquals(numPartitions, actualRecords.getNumPartitions());
     List<HoodieRecord> collectedActualRecords = actualRecords.collect();
     if (isGloballySorted) {
diff --git a/hudi-client/src/test/java/org/apache/hudi/index/TestHoodieIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndex.java
similarity index 87%
rename from hudi-client/src/test/java/org/apache/hudi/index/TestHoodieIndex.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndex.java
index aefad87..9de36c6 100644
--- a/hudi-client/src/test/java/org/apache/hudi/index/TestHoodieIndex.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndex.java
@@ -35,7 +35,9 @@
 import org.apache.hudi.config.HoodieStorageConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.index.HoodieIndex.IndexType;
+import org.apache.hudi.table.HoodieSparkTable;
 import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.testutils.Assertions;
 import org.apache.hudi.testutils.HoodieClientTestHarness;
 import org.apache.hudi.testutils.HoodieWriteableTestTable;
 import org.apache.hudi.testutils.MetadataMergeWriteStatus;
@@ -60,7 +62,6 @@
 import scala.Tuple2;
 
 import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource;
-import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertTrue;
@@ -99,27 +100,27 @@
     JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
 
     metaClient = HoodieTableMetaClient.reload(metaClient);
-    HoodieTable hoodieTable = HoodieTable.create(metaClient, config, jsc.hadoopConfiguration());
+    HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
 
     // Test tagLocation without any entries in index
-    JavaRDD<HoodieRecord> javaRDD = index.tagLocation(writeRecords, jsc, hoodieTable);
+    JavaRDD<HoodieRecord> javaRDD = (JavaRDD<HoodieRecord>) index.tagLocation(writeRecords, context, hoodieTable);
     assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 0);
 
     // Insert totalRecords records
     writeClient.startCommitWithTime(newCommitTime);
     JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
-    assertNoWriteErrors(writeStatues.collect());
+    Assertions.assertNoWriteErrors(writeStatues.collect());
 
     // Now tagLocation for these records, index should not tag them since it was a failed
     // commit
-    javaRDD = index.tagLocation(writeRecords, jsc, hoodieTable);
+    javaRDD = (JavaRDD<HoodieRecord>) index.tagLocation(writeRecords, context, hoodieTable);
     assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 0);
     // Now commit this & update location of records inserted and validate no errors
     writeClient.commit(newCommitTime, writeStatues);
     // Now tagLocation for these records, index should tag them correctly
     metaClient = HoodieTableMetaClient.reload(metaClient);
-    hoodieTable = HoodieTable.create(metaClient, config, jsc.hadoopConfiguration());
-    javaRDD = index.tagLocation(writeRecords, jsc, hoodieTable);
+    hoodieTable = HoodieSparkTable.create(config, context, metaClient);
+    javaRDD = (JavaRDD<HoodieRecord>) index.tagLocation(writeRecords, context, hoodieTable);
     Map<String, String> recordKeyToPartitionPathMap = new HashMap();
     List<HoodieRecord> hoodieRecords = writeRecords.collect();
     hoodieRecords.forEach(entry -> recordKeyToPartitionPathMap.put(entry.getRecordKey(), entry.getPartitionPath()));
@@ -131,7 +132,7 @@
     javaRDD.foreach(entry -> assertEquals(recordKeyToPartitionPathMap.get(entry.getRecordKey()), entry.getPartitionPath(), "PartitionPath mismatch"));
 
     JavaRDD<HoodieKey> hoodieKeyJavaRDD = writeRecords.map(entry -> entry.getKey());
-    JavaPairRDD<HoodieKey, Option<Pair<String, String>>> recordLocations = index.fetchRecordLocation(hoodieKeyJavaRDD, jsc, hoodieTable);
+    JavaPairRDD<HoodieKey, Option<Pair<String, String>>> recordLocations = getRecordLocations(hoodieKeyJavaRDD, hoodieTable);
     List<HoodieKey> hoodieKeys = hoodieKeyJavaRDD.collect();
     assertEquals(totalRecords, recordLocations.collect().size());
     assertEquals(totalRecords, recordLocations.map(record -> record._1).distinct().count());
@@ -148,11 +149,11 @@
     List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, totalRecords);
     JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
 
-    HoodieTable hoodieTable = HoodieTable.create(metaClient, config, jsc.hadoopConfiguration());
+    HoodieSparkTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
 
     writeClient.startCommitWithTime(newCommitTime);
     JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
-    JavaRDD<HoodieRecord> javaRDD1 = index.tagLocation(writeRecords, jsc, hoodieTable);
+    JavaRDD<HoodieRecord> javaRDD1 = (JavaRDD<HoodieRecord>) index.tagLocation(writeRecords, context, hoodieTable);
 
     // Duplicate upsert and ensure correctness is maintained
     // We are trying to approximately imitate the case when the RDD is recomputed. For RDD creating, driver code is not
@@ -161,14 +162,14 @@
     metaClient.getFs().delete(new Path(metaClient.getMetaPath(), "001.inflight"));
 
     writeClient.upsert(writeRecords, newCommitTime);
-    assertNoWriteErrors(writeStatues.collect());
+    Assertions.assertNoWriteErrors(writeStatues.collect());
 
     // Now commit this & update location of records inserted and validate no errors
     writeClient.commit(newCommitTime, writeStatues);
     // Now tagLocation for these records, hbaseIndex should tag them correctly
     metaClient = HoodieTableMetaClient.reload(metaClient);
-    hoodieTable = HoodieTable.create(metaClient, config, jsc.hadoopConfiguration());
-    JavaRDD<HoodieRecord> javaRDD = index.tagLocation(writeRecords, jsc, hoodieTable);
+    hoodieTable = HoodieSparkTable.create(config, context, metaClient);
+    JavaRDD<HoodieRecord> javaRDD = (JavaRDD<HoodieRecord>) index.tagLocation(writeRecords, context, hoodieTable);
 
     Map<String, String> recordKeyToPartitionPathMap = new HashMap();
     List<HoodieRecord> hoodieRecords = writeRecords.collect();
@@ -181,7 +182,7 @@
     javaRDD.foreach(entry -> assertEquals(recordKeyToPartitionPathMap.get(entry.getRecordKey()), entry.getPartitionPath(), "PartitionPath mismatch"));
 
     JavaRDD<HoodieKey> hoodieKeyJavaRDD = writeRecords.map(entry -> entry.getKey());
-    JavaPairRDD<HoodieKey, Option<Pair<String, String>>> recordLocations = index.fetchRecordLocation(hoodieKeyJavaRDD, jsc, hoodieTable);
+    JavaPairRDD<HoodieKey, Option<Pair<String, String>>> recordLocations = getRecordLocations(hoodieKeyJavaRDD, hoodieTable);
     List<HoodieKey> hoodieKeys = hoodieKeyJavaRDD.collect();
     assertEquals(totalRecords, recordLocations.collect().size());
     assertEquals(totalRecords, recordLocations.map(record -> record._1).distinct().count());
@@ -201,14 +202,14 @@
 
     // Insert 200 records
     JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
-    assertNoWriteErrors(writeStatues.collect());
+    Assertions.assertNoWriteErrors(writeStatues.collect());
 
     // commit this upsert
     writeClient.commit(newCommitTime, writeStatues);
-    HoodieTable hoodieTable = HoodieTable.create(metaClient, config, jsc.hadoopConfiguration());
+    HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
 
     // Now tagLocation for these records, hbaseIndex should tag them
-    JavaRDD<HoodieRecord> javaRDD = index.tagLocation(writeRecords, jsc, hoodieTable);
+    JavaRDD<HoodieRecord> javaRDD = (JavaRDD<HoodieRecord>) index.tagLocation(writeRecords, context, hoodieTable);
     assert (javaRDD.filter(HoodieRecord::isCurrentLocationKnown).collect().size() == totalRecords);
 
     // check tagged records are tagged with correct fileIds
@@ -221,7 +222,7 @@
     hoodieRecords.forEach(entry -> recordKeyToPartitionPathMap.put(entry.getRecordKey(), entry.getPartitionPath()));
 
     JavaRDD<HoodieKey> hoodieKeyJavaRDD = writeRecords.map(entry -> entry.getKey());
-    JavaPairRDD<HoodieKey, Option<Pair<String, String>>> recordLocations = index.fetchRecordLocation(hoodieKeyJavaRDD, jsc, hoodieTable);
+    JavaPairRDD<HoodieKey, Option<Pair<String, String>>> recordLocations = getRecordLocations(hoodieKeyJavaRDD, hoodieTable);
     List<HoodieKey> hoodieKeys = hoodieKeyJavaRDD.collect();
     assertEquals(totalRecords, recordLocations.collect().size());
     assertEquals(totalRecords, recordLocations.map(record -> record._1).distinct().count());
@@ -233,10 +234,10 @@
     // Rollback the last commit
     writeClient.rollback(newCommitTime);
 
-    hoodieTable = HoodieTable.create(metaClient, config, jsc.hadoopConfiguration());
+    hoodieTable = HoodieSparkTable.create(config, context, metaClient);
     // Now tagLocation for these records, hbaseIndex should not tag them since it was a rolled
     // back commit
-    javaRDD = index.tagLocation(writeRecords, jsc, hoodieTable);
+    javaRDD = (JavaRDD<HoodieRecord>) index.tagLocation(writeRecords, context, hoodieTable);
     assert (javaRDD.filter(HoodieRecord::isCurrentLocationKnown).collect().size() == 0);
     assert (javaRDD.filter(record -> record.getCurrentLocation() != null).collect().size() == 0);
   }
@@ -269,9 +270,9 @@
         new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
     JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Arrays.asList(record1, record2, record3, record4));
 
-    HoodieTable hoodieTable = HoodieTable.create(metaClient, config, jsc.hadoopConfiguration());
+    HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
 
-    JavaRDD<HoodieRecord> taggedRecordRDD = index.tagLocation(recordRDD, jsc, hoodieTable);
+    JavaRDD<HoodieRecord> taggedRecordRDD = (JavaRDD<HoodieRecord>) index.tagLocation(recordRDD, context, hoodieTable);
 
     // Should not find any files
     for (HoodieRecord record : taggedRecordRDD.collect()) {
@@ -286,9 +287,9 @@
 
     // We do the tag again
     metaClient = HoodieTableMetaClient.reload(metaClient);
-    hoodieTable = HoodieTable.create(metaClient, config, jsc.hadoopConfiguration());
+    hoodieTable = HoodieSparkTable.create(config, context, metaClient);
 
-    taggedRecordRDD = index.tagLocation(recordRDD, jsc, hoodieTable);
+    taggedRecordRDD = (JavaRDD<HoodieRecord>) index.tagLocation(recordRDD, context, hoodieTable);
 
     // Check results
     for (HoodieRecord record : taggedRecordRDD.collect()) {
@@ -305,8 +306,7 @@
       }
     }
 
-    JavaPairRDD<HoodieKey, Option<Pair<String, String>>> recordLocations = index.fetchRecordLocation(recordRDD.map(entry -> entry.getKey()), jsc, hoodieTable);
-
+    JavaPairRDD<HoodieKey, Option<Pair<String, String>>> recordLocations = getRecordLocations(recordRDD.map(HoodieRecord::getKey), hoodieTable);
     for (Tuple2<HoodieKey, Option<Pair<String, String>>> entry : recordLocations.collect()) {
       if (entry._1.getRecordKey().equals(rowKey1)) {
         assertTrue(entry._2.isPresent(), "Row1 should have been present ");
@@ -336,7 +336,7 @@
             .build()).build();
     writeClient = getHoodieWriteClient(config);
     index = writeClient.getIndex();
-    HoodieTable hoodieTable = HoodieTable.create(metaClient, config, jsc.hadoopConfiguration());
+    HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
     HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, SCHEMA);
     final String p1 = "2016/01/31";
     final String p2 = "2016/02/28";
@@ -379,7 +379,7 @@
 
     // test against incoming record with a different partition
     JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Collections.singletonList(incomingRecord));
-    JavaRDD<HoodieRecord> taggedRecordRDD = index.tagLocation(recordRDD, jsc, hoodieTable);
+    JavaRDD<HoodieRecord> taggedRecordRDD = (JavaRDD<HoodieRecord>) index.tagLocation(recordRDD, context, hoodieTable);
 
     assertEquals(2, taggedRecordRDD.count());
     for (HoodieRecord record : taggedRecordRDD.collect()) {
@@ -400,7 +400,7 @@
     // test against incoming record with the same partition
     JavaRDD<HoodieRecord> recordRDDSamePartition = jsc
         .parallelize(Collections.singletonList(incomingRecordSamePartition));
-    JavaRDD<HoodieRecord> taggedRecordRDDSamePartition = index.tagLocation(recordRDDSamePartition, jsc, hoodieTable);
+    JavaRDD<HoodieRecord> taggedRecordRDDSamePartition = (JavaRDD<HoodieRecord>) index.tagLocation(recordRDDSamePartition, context, hoodieTable);
 
     assertEquals(1, taggedRecordRDDSamePartition.count());
     HoodieRecord record = taggedRecordRDDSamePartition.first();
@@ -422,4 +422,12 @@
             .withStorageType(FileSystemViewStorageType.EMBEDDED_KV_STORE).build());
   }
 
+  private JavaPairRDD<HoodieKey, Option<Pair<String, String>>> getRecordLocations(JavaRDD<HoodieKey> keyRDD, HoodieTable hoodieTable) {
+    JavaRDD<HoodieRecord> recordRDD = (JavaRDD<HoodieRecord>) index.tagLocation(
+        keyRDD.map(k -> new HoodieRecord(k, new EmptyHoodieRecordPayload())), context, hoodieTable);
+    return recordRDD.mapToPair(hr -> new Tuple2<>(hr.getKey(), hr.isCurrentLocationKnown()
+        ? Option.of(Pair.of(hr.getPartitionPath(), hr.getCurrentLocation().getFileId()))
+        : Option.empty())
+    );
+  }
 }
diff --git a/hudi-client/src/test/java/org/apache/hudi/index/TestHoodieIndexConfigs.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndexConfigs.java
similarity index 78%
rename from hudi-client/src/test/java/org/apache/hudi/index/TestHoodieIndexConfigs.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndexConfigs.java
index a68a872..9175ebd 100644
--- a/hudi-client/src/test/java/org/apache/hudi/index/TestHoodieIndexConfigs.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/TestHoodieIndexConfigs.java
@@ -20,26 +20,23 @@
 package org.apache.hudi.index;
 
 import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieEngineContext;
 import org.apache.hudi.common.model.HoodieKey;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieRecordPayload;
-import org.apache.hudi.common.util.Option;
-import org.apache.hudi.common.util.collection.Pair;
 import org.apache.hudi.config.HoodieHBaseIndexConfig;
 import org.apache.hudi.config.HoodieIndexConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.exception.HoodieException;
 import org.apache.hudi.exception.HoodieIndexException;
 import org.apache.hudi.index.HoodieIndex.IndexType;
-import org.apache.hudi.index.bloom.HoodieBloomIndex;
-import org.apache.hudi.index.bloom.HoodieGlobalBloomIndex;
-import org.apache.hudi.index.hbase.HBaseIndex;
-import org.apache.hudi.index.simple.HoodieSimpleIndex;
+import org.apache.hudi.index.bloom.SparkHoodieBloomIndex;
+import org.apache.hudi.index.bloom.SparkHoodieGlobalBloomIndex;
+import org.apache.hudi.index.hbase.SparkHoodieHBaseIndex;
+import org.apache.hudi.index.simple.SparkHoodieSimpleIndex;
 import org.apache.hudi.table.HoodieTable;
 
-import org.apache.spark.api.java.JavaPairRDD;
 import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.io.TempDir;
@@ -51,6 +48,7 @@
 import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
+@SuppressWarnings("checkstyle:LineLength")
 public class TestHoodieIndexConfigs {
 
   private String basePath;
@@ -70,29 +68,29 @@
       case INMEMORY:
         config = clientConfigBuilder.withPath(basePath)
             .withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build();
-        assertTrue(HoodieIndex.createIndex(config) instanceof InMemoryHashIndex);
+        assertTrue(SparkHoodieIndex.createIndex(config) instanceof SparkInMemoryHashIndex);
         break;
       case BLOOM:
         config = clientConfigBuilder.withPath(basePath)
             .withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.BLOOM).build()).build();
-        assertTrue(HoodieIndex.createIndex(config) instanceof HoodieBloomIndex);
+        assertTrue(SparkHoodieIndex.createIndex(config) instanceof SparkHoodieBloomIndex);
         break;
       case GLOBAL_BLOOM:
         config = clientConfigBuilder.withPath(basePath)
             .withIndexConfig(indexConfigBuilder.withIndexType(IndexType.GLOBAL_BLOOM).build()).build();
-        assertTrue(HoodieIndex.createIndex(config) instanceof HoodieGlobalBloomIndex);
+        assertTrue(SparkHoodieIndex.createIndex(config) instanceof SparkHoodieGlobalBloomIndex);
         break;
       case SIMPLE:
         config = clientConfigBuilder.withPath(basePath)
             .withIndexConfig(indexConfigBuilder.withIndexType(IndexType.SIMPLE).build()).build();
-        assertTrue(HoodieIndex.createIndex(config) instanceof HoodieSimpleIndex);
+        assertTrue(SparkHoodieIndex.createIndex(config) instanceof SparkHoodieSimpleIndex);
         break;
       case HBASE:
         config = clientConfigBuilder.withPath(basePath)
             .withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.HBASE)
                 .withHBaseIndexConfig(new HoodieHBaseIndexConfig.Builder().build()).build())
             .build();
-        assertTrue(HoodieIndex.createIndex(config) instanceof HBaseIndex);
+        assertTrue(SparkHoodieIndex.createIndex(config) instanceof SparkHoodieHBaseIndex);
         break;
       default:
         // no -op. just for checkstyle errors
@@ -105,7 +103,7 @@
     HoodieIndexConfig.Builder indexConfigBuilder = HoodieIndexConfig.newBuilder();
     HoodieWriteConfig config = clientConfigBuilder.withPath(basePath)
         .withIndexConfig(indexConfigBuilder.withIndexClass(DummyHoodieIndex.class.getName()).build()).build();
-    assertTrue(HoodieIndex.createIndex(config) instanceof DummyHoodieIndex);
+    assertTrue(SparkHoodieIndex.createIndex(config) instanceof DummyHoodieIndex);
   }
 
   @Test
@@ -115,36 +113,35 @@
     final HoodieWriteConfig config1 = clientConfigBuilder.withPath(basePath)
         .withIndexConfig(indexConfigBuilder.withIndexClass(IndexWithConstructor.class.getName()).build()).build();
     final Throwable thrown1 = assertThrows(HoodieException.class, () -> {
-      HoodieIndex.createIndex(config1);
+      SparkHoodieIndex.createIndex(config1);
     }, "exception is expected");
     assertTrue(thrown1.getMessage().contains("is not a subclass of HoodieIndex"));
 
     final HoodieWriteConfig config2 = clientConfigBuilder.withPath(basePath)
         .withIndexConfig(indexConfigBuilder.withIndexClass(IndexWithoutConstructor.class.getName()).build()).build();
     final Throwable thrown2 = assertThrows(HoodieException.class, () -> {
-      HoodieIndex.createIndex(config2);
+      SparkHoodieIndex.createIndex(config2);
     }, "exception is expected");
     assertTrue(thrown2.getMessage().contains("Unable to instantiate class"));
   }
 
-  public static class DummyHoodieIndex<T extends HoodieRecordPayload<T>> extends HoodieIndex<T> {
+  public static class DummyHoodieIndex<T extends HoodieRecordPayload<T>> extends SparkHoodieIndex<T> {
 
     public DummyHoodieIndex(HoodieWriteConfig config) {
       super(config);
     }
 
     @Override
-    public JavaPairRDD<HoodieKey, Option<Pair<String, String>>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys, JavaSparkContext jsc, HoodieTable<T> hoodieTable) {
+    public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
+                                               HoodieEngineContext context,
+                                               HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) throws HoodieIndexException {
       return null;
     }
 
     @Override
-    public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, JavaSparkContext jsc, HoodieTable<T> hoodieTable) throws HoodieIndexException {
-      return null;
-    }
-
-    @Override
-    public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, JavaSparkContext jsc, HoodieTable<T> hoodieTable) throws HoodieIndexException {
+    public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> records,
+                                                HoodieEngineContext context,
+                                                HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) throws HoodieIndexException {
       return null;
     }
 
diff --git a/hudi-client/src/test/java/org/apache/hudi/index/bloom/TestBucketizedBloomCheckPartitioner.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestBucketizedBloomCheckPartitioner.java
similarity index 100%
rename from hudi-client/src/test/java/org/apache/hudi/index/bloom/TestBucketizedBloomCheckPartitioner.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestBucketizedBloomCheckPartitioner.java
diff --git a/hudi-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java
similarity index 90%
rename from hudi-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java
index 4584324..2d091a0 100644
--- a/hudi-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieBloomIndex.java
@@ -30,6 +30,7 @@
 import org.apache.hudi.config.HoodieIndexConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.io.HoodieKeyLookupHandle;
+import org.apache.hudi.table.HoodieSparkTable;
 import org.apache.hudi.table.HoodieTable;
 import org.apache.hudi.testutils.HoodieClientTestHarness;
 import org.apache.hudi.testutils.HoodieWriteableTestTable;
@@ -102,8 +103,8 @@
   @MethodSource("configParams")
   public void testLoadInvolvedFiles(boolean rangePruning, boolean treeFiltering, boolean bucketizedChecking) throws Exception {
     HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking);
-    HoodieBloomIndex index = new HoodieBloomIndex(config);
-    HoodieTable hoodieTable = HoodieTable.create(metaClient, config, hadoopConf);
+    SparkHoodieBloomIndex index = new SparkHoodieBloomIndex(config);
+    HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
     HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, SCHEMA);
 
     // Create some partitions, and put some files
@@ -130,7 +131,7 @@
         new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
 
     List<String> partitions = Arrays.asList("2016/01/21", "2016/04/01", "2015/03/12");
-    List<Tuple2<String, BloomIndexFileInfo>> filesList = index.loadInvolvedFiles(partitions, jsc, hoodieTable);
+    List<Tuple2<String, BloomIndexFileInfo>> filesList = index.loadInvolvedFiles(partitions, context, hoodieTable);
     // Still 0, as no valid commit
     assertEquals(0, filesList.size());
 
@@ -139,7 +140,7 @@
         .withInserts("2015/03/12", "3", record1)
         .withInserts("2015/03/12", "4", record2, record3, record4);
 
-    filesList = index.loadInvolvedFiles(partitions, jsc, hoodieTable);
+    filesList = index.loadInvolvedFiles(partitions, context, hoodieTable);
     assertEquals(4, filesList.size());
 
     if (rangePruning) {
@@ -166,7 +167,7 @@
   @MethodSource("configParams")
   public void testRangePruning(boolean rangePruning, boolean treeFiltering, boolean bucketizedChecking) {
     HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking);
-    HoodieBloomIndex index = new HoodieBloomIndex(config);
+    SparkHoodieBloomIndex index = new SparkHoodieBloomIndex(config);
 
     final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo = new HashMap<>();
     partitionToFileIndexInfo.put("2017/10/22",
@@ -236,7 +237,7 @@
         Arrays.asList(record1.getRecordKey(), record2.getRecordKey(), record3.getRecordKey(), record4.getRecordKey());
 
     HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
-    HoodieTable table = HoodieTable.create(metaClient, config, hadoopConf);
+    HoodieSparkTable table = HoodieSparkTable.create(config, context, metaClient);
     HoodieKeyLookupHandle keyHandle = new HoodieKeyLookupHandle<>(config, table, Pair.of(partition, fileId));
     List<String> results = keyHandle.checkCandidatesAgainstFile(hadoopConf, uuids,
         new Path(Paths.get(basePath, partition, filename).toString()));
@@ -258,13 +259,13 @@
     // Also create the metadata and config
     HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking);
     metaClient = HoodieTableMetaClient.reload(metaClient);
-    HoodieTable table = HoodieTable.create(metaClient, config, hadoopConf);
+    HoodieSparkTable table = HoodieSparkTable.create(config, context, metaClient);
 
     // Let's tag
-    HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config);
+    SparkHoodieBloomIndex bloomIndex = new SparkHoodieBloomIndex(config);
 
     assertDoesNotThrow(() -> {
-      bloomIndex.tagLocation(recordRDD, jsc, table);
+      bloomIndex.tagLocation(recordRDD, context, table);
     }, "EmptyRDD should not result in IllegalArgumentException: Positive number of slices required");
   }
 
@@ -296,12 +297,12 @@
 
     // Also create the metadata and config
     HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking);
-    HoodieTable hoodieTable = HoodieTable.create(metaClient, config, hadoopConf);
+    HoodieSparkTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
     HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, SCHEMA);
 
     // Let's tag
-    HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config);
-    JavaRDD<HoodieRecord> taggedRecordRDD = bloomIndex.tagLocation(recordRDD, jsc, hoodieTable);
+    SparkHoodieBloomIndex bloomIndex = new SparkHoodieBloomIndex(config);
+    JavaRDD<HoodieRecord> taggedRecordRDD = bloomIndex.tagLocation(recordRDD, context, hoodieTable);
 
     // Should not find any files
     for (HoodieRecord record : taggedRecordRDD.collect()) {
@@ -314,7 +315,7 @@
     String fileId3 = testTable.addCommit("003").withInserts("2015/01/31", record4);
 
     // We do the tag again
-    taggedRecordRDD = bloomIndex.tagLocation(recordRDD, jsc, HoodieTable.create(metaClient, config, hadoopConf));
+    taggedRecordRDD = bloomIndex.tagLocation(recordRDD, context, HoodieSparkTable.create(config, context, metaClient));
 
     // Check results
     for (HoodieRecord record : taggedRecordRDD.collect()) {
@@ -361,16 +362,20 @@
 
     // Also create the metadata and config
     HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking);
-    HoodieTable hoodieTable = HoodieTable.create(metaClient, config, hadoopConf);
+    HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
     HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, SCHEMA);
 
     // Let's tag
-    HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config);
-    JavaPairRDD<HoodieKey, Option<Pair<String, String>>> taggedRecordRDD =
-        bloomIndex.fetchRecordLocation(keysRDD, jsc, hoodieTable);
+    SparkHoodieBloomIndex bloomIndex = new SparkHoodieBloomIndex(config);
+    JavaRDD<HoodieRecord> taggedRecords = bloomIndex.tagLocation(keysRDD.map(k -> new HoodieRecord(k, null)), context, hoodieTable);
+    JavaPairRDD<HoodieKey, Option<Pair<String, String>>> recordLocationsRDD = taggedRecords
+        .mapToPair(hr -> new Tuple2<>(hr.getKey(), hr.isCurrentLocationKnown()
+            ? Option.of(Pair.of(hr.getPartitionPath(), hr.getCurrentLocation().getFileId()))
+            : Option.empty())
+        );
 
     // Should not find any files
-    for (Tuple2<HoodieKey, Option<Pair<String, String>>> record : taggedRecordRDD.collect()) {
+    for (Tuple2<HoodieKey, Option<Pair<String, String>>> record : recordLocationsRDD.collect()) {
       assertTrue(!record._2.isPresent());
     }
 
@@ -381,11 +386,16 @@
 
     // We do the tag again
     metaClient = HoodieTableMetaClient.reload(metaClient);
-    hoodieTable = HoodieTable.create(metaClient, config, hadoopConf);
-    taggedRecordRDD = bloomIndex.fetchRecordLocation(keysRDD, jsc, hoodieTable);
+    hoodieTable = HoodieSparkTable.create(config, context, metaClient);
+    taggedRecords = bloomIndex.tagLocation(keysRDD.map(k -> new HoodieRecord(k, null)), context, hoodieTable);
+    recordLocationsRDD = taggedRecords
+        .mapToPair(hr -> new Tuple2<>(hr.getKey(), hr.isCurrentLocationKnown()
+            ? Option.of(Pair.of(hr.getPartitionPath(), hr.getCurrentLocation().getFileId()))
+            : Option.empty())
+        );
 
     // Check results
-    for (Tuple2<HoodieKey, Option<Pair<String, String>>> record : taggedRecordRDD.collect()) {
+    for (Tuple2<HoodieKey, Option<Pair<String, String>>> record : recordLocationsRDD.collect()) {
       if (record._1.getRecordKey().equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0")) {
         assertTrue(record._2.isPresent());
         assertEquals(fileId1, record._2.get().getRight());
@@ -431,10 +441,10 @@
     JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Arrays.asList(record1, record2));
     HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking);
     metaClient = HoodieTableMetaClient.reload(metaClient);
-    HoodieTable table = HoodieTable.create(metaClient, config, hadoopConf);
+    HoodieTable table = HoodieSparkTable.create(config, context, metaClient);
 
-    HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config);
-    JavaRDD<HoodieRecord> taggedRecordRDD = bloomIndex.tagLocation(recordRDD, jsc, table);
+    SparkHoodieBloomIndex bloomIndex = new SparkHoodieBloomIndex(config);
+    JavaRDD<HoodieRecord> taggedRecordRDD = bloomIndex.tagLocation(recordRDD, context, table);
 
     // Check results
     for (HoodieRecord record : taggedRecordRDD.collect()) {
diff --git a/hudi-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieGlobalBloomIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieGlobalBloomIndex.java
similarity index 94%
rename from hudi-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieGlobalBloomIndex.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieGlobalBloomIndex.java
index 197a3bb..2f68a03 100644
--- a/hudi-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieGlobalBloomIndex.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestHoodieGlobalBloomIndex.java
@@ -24,6 +24,7 @@
 import org.apache.hudi.common.testutils.RawTripTestPayload;
 import org.apache.hudi.config.HoodieIndexConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.table.HoodieSparkTable;
 import org.apache.hudi.table.HoodieTable;
 import org.apache.hudi.testutils.HoodieClientTestHarness;
 import org.apache.hudi.testutils.HoodieWriteableTestTable;
@@ -73,8 +74,8 @@
   @Test
   public void testLoadInvolvedFiles() throws Exception {
     HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
-    HoodieGlobalBloomIndex index = new HoodieGlobalBloomIndex(config);
-    HoodieTable hoodieTable = HoodieTable.create(metaClient, config, hadoopConf);
+    SparkHoodieGlobalBloomIndex index = new SparkHoodieGlobalBloomIndex(config);
+    HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
     HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, SCHEMA);
 
     // Create some partitions, and put some files, along with the meta file
@@ -103,7 +104,7 @@
     // intentionally missed the partition "2015/03/12" to see if the GlobalBloomIndex can pick it up
     List<String> partitions = Arrays.asList("2016/01/21", "2016/04/01");
     // partitions will NOT be respected by this loadInvolvedFiles(...) call
-    List<Tuple2<String, BloomIndexFileInfo>> filesList = index.loadInvolvedFiles(partitions, jsc, hoodieTable);
+    List<Tuple2<String, BloomIndexFileInfo>> filesList = index.loadInvolvedFiles(partitions, context, hoodieTable);
     // Still 0, as no valid commit
     assertEquals(0, filesList.size());
 
@@ -112,7 +113,7 @@
         .withInserts("2015/03/12", "3", record1)
         .withInserts("2015/03/12", "4", record2, record3, record4);
 
-    filesList = index.loadInvolvedFiles(partitions, jsc, hoodieTable);
+    filesList = index.loadInvolvedFiles(partitions, context, hoodieTable);
     assertEquals(4, filesList.size());
 
     Map<String, BloomIndexFileInfo> filesMap = toFileMap(filesList);
@@ -137,7 +138,7 @@
   public void testExplodeRecordRDDWithFileComparisons() {
 
     HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
-    HoodieGlobalBloomIndex index = new HoodieGlobalBloomIndex(config);
+    SparkHoodieGlobalBloomIndex index = new SparkHoodieGlobalBloomIndex(config);
 
     final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo = new HashMap<>();
     partitionToFileIndexInfo.put("2017/10/22", Arrays.asList(new BloomIndexFileInfo("f1"),
@@ -177,8 +178,8 @@
   @Test
   public void testTagLocation() throws Exception {
     HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
-    HoodieGlobalBloomIndex index = new HoodieGlobalBloomIndex(config);
-    HoodieTable hoodieTable = HoodieTable.create(metaClient, config, hadoopConf);
+    SparkHoodieGlobalBloomIndex index = new SparkHoodieGlobalBloomIndex(config);
+    HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
     HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, SCHEMA);
 
     // Create some partitions, and put some files, along with the meta file
@@ -222,7 +223,7 @@
     String fileId4 = testTable.addCommit("4000").withInserts("2015/03/12", record4);
 
     // partitions will NOT be respected by this loadInvolvedFiles(...) call
-    JavaRDD<HoodieRecord> taggedRecordRDD = index.tagLocation(recordRDD, jsc, hoodieTable);
+    JavaRDD<HoodieRecord> taggedRecordRDD = index.tagLocation(recordRDD, context, hoodieTable);
 
     for (HoodieRecord record : taggedRecordRDD.collect()) {
       switch (record.getRecordKey()) {
@@ -258,8 +259,8 @@
         .withPath(basePath)
         .withIndexConfig(HoodieIndexConfig.newBuilder().withBloomIndexUpdatePartitionPath(true).build())
         .build();
-    HoodieGlobalBloomIndex index = new HoodieGlobalBloomIndex(config);
-    HoodieTable hoodieTable = HoodieTable.create(metaClient, config, jsc.hadoopConfiguration());
+    SparkHoodieGlobalBloomIndex index = new SparkHoodieGlobalBloomIndex(config);
+    HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
     HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, SCHEMA);
     final String p1 = "2016/01/31";
     final String p2 = "2016/02/28";
@@ -302,7 +303,7 @@
 
     // test against incoming record with a different partition
     JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Collections.singletonList(incomingRecord));
-    JavaRDD<HoodieRecord> taggedRecordRDD = index.tagLocation(recordRDD, jsc, hoodieTable);
+    JavaRDD<HoodieRecord> taggedRecordRDD = index.tagLocation(recordRDD, context, hoodieTable);
 
     assertEquals(2, taggedRecordRDD.count());
     for (HoodieRecord record : taggedRecordRDD.collect()) {
@@ -323,7 +324,7 @@
     // test against incoming record with the same partition
     JavaRDD<HoodieRecord> recordRDDSamePartition = jsc
         .parallelize(Collections.singletonList(incomingRecordSamePartition));
-    JavaRDD<HoodieRecord> taggedRecordRDDSamePartition = index.tagLocation(recordRDDSamePartition, jsc, hoodieTable);
+    JavaRDD<HoodieRecord> taggedRecordRDDSamePartition = index.tagLocation(recordRDDSamePartition, context, hoodieTable);
 
     assertEquals(1, taggedRecordRDDSamePartition.count());
     HoodieRecord record = taggedRecordRDDSamePartition.first();
diff --git a/hudi-client/src/test/java/org/apache/hudi/index/bloom/TestKeyRangeLookupTree.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestKeyRangeLookupTree.java
similarity index 100%
rename from hudi-client/src/test/java/org/apache/hudi/index/bloom/TestKeyRangeLookupTree.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/bloom/TestKeyRangeLookupTree.java
diff --git a/hudi-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseIndex.java
similarity index 88%
rename from hudi-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseIndex.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseIndex.java
index db8ca52..b858516 100644
--- a/hudi-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseIndex.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseIndex.java
@@ -18,7 +18,7 @@
 
 package org.apache.hudi.index.hbase;
 
-import org.apache.hudi.client.HoodieWriteClient;
+import org.apache.hudi.client.SparkRDDWriteClient;
 import org.apache.hudi.client.WriteStatus;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieTableType;
@@ -33,6 +33,7 @@
 import org.apache.hudi.config.HoodieStorageConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.index.HoodieIndex;
+import org.apache.hudi.table.HoodieSparkTable;
 import org.apache.hudi.table.HoodieTable;
 import org.apache.hudi.testutils.FunctionalTestHarness;
 
@@ -136,13 +137,13 @@
 
     // Load to memory
     HoodieWriteConfig config = getConfig();
-    HBaseIndex index = new HBaseIndex(config);
-    try (HoodieWriteClient writeClient = getHoodieWriteClient(config);) {
+    SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config);
+    try (SparkRDDWriteClient writeClient = getHoodieWriteClient(config);) {
       metaClient = HoodieTableMetaClient.reload(metaClient);
-      HoodieTable hoodieTable = HoodieTable.create(metaClient, config, hadoopConf);
+      HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
 
       // Test tagLocation without any entries in index
-      JavaRDD<HoodieRecord> records1 = index.tagLocation(writeRecords, jsc(), hoodieTable);
+      JavaRDD<HoodieRecord> records1 = index.tagLocation(writeRecords, context(), hoodieTable);
       assertEquals(0, records1.filter(record -> record.isCurrentLocationKnown()).count());
 
       // Insert 200 records
@@ -151,15 +152,15 @@
       assertNoWriteErrors(writeStatues.collect());
 
       // Now tagLocation for these records, hbaseIndex should not tag them since commit never occurred
-      JavaRDD<HoodieRecord> records2 = index.tagLocation(writeRecords, jsc(), hoodieTable);
+      JavaRDD<HoodieRecord> records2 = index.tagLocation(writeRecords, context(), hoodieTable);
       assertEquals(0, records2.filter(record -> record.isCurrentLocationKnown()).count());
 
       // Now commit this & update location of records inserted and validate no errors
       writeClient.commit(newCommitTime, writeStatues);
       // Now tagLocation for these records, hbaseIndex should tag them correctly
       metaClient = HoodieTableMetaClient.reload(metaClient);
-      hoodieTable = HoodieTable.create(metaClient, config, hadoopConf);
-      List<HoodieRecord> records3 = index.tagLocation(writeRecords, jsc(), hoodieTable).collect();
+      hoodieTable = HoodieSparkTable.create(config, context, metaClient);
+      List<HoodieRecord> records3 = index.tagLocation(writeRecords, context(), hoodieTable).collect();
       assertEquals(numRecords, records3.stream().filter(record -> record.isCurrentLocationKnown()).count());
       assertEquals(numRecords, records3.stream().map(record -> record.getKey().getRecordKey()).distinct().count());
       assertEquals(numRecords, records3.stream().filter(record -> (record.getCurrentLocation() != null
@@ -176,14 +177,14 @@
 
     // Load to memory
     HoodieWriteConfig config = getConfig();
-    HBaseIndex index = new HBaseIndex(config);
-    HoodieWriteClient writeClient = getHoodieWriteClient(config);
+    SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config);
+    SparkRDDWriteClient writeClient = getHoodieWriteClient(config);
     writeClient.startCommitWithTime(newCommitTime);
     metaClient = HoodieTableMetaClient.reload(metaClient);
-    HoodieTable hoodieTable = HoodieTable.create(metaClient, config, hadoopConf);
+    HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
 
     JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
-    index.tagLocation(writeRecords, jsc(), hoodieTable);
+    index.tagLocation(writeRecords, context(), hoodieTable);
 
     // Duplicate upsert and ensure correctness is maintained
     // We are trying to approximately imitate the case when the RDD is recomputed. For RDD creating, driver code is not
@@ -198,8 +199,8 @@
     writeClient.commit(newCommitTime, writeStatues);
     // Now tagLocation for these records, hbaseIndex should tag them correctly
     metaClient = HoodieTableMetaClient.reload(metaClient);
-    hoodieTable = HoodieTable.create(metaClient, config, hadoopConf);
-    List<HoodieRecord> taggedRecords = index.tagLocation(writeRecords, jsc(), hoodieTable).collect();
+    hoodieTable = HoodieSparkTable.create(config, context, metaClient);
+    List<HoodieRecord> taggedRecords = index.tagLocation(writeRecords, context(), hoodieTable).collect();
     assertEquals(numRecords, taggedRecords.stream().filter(HoodieRecord::isCurrentLocationKnown).count());
     assertEquals(numRecords, taggedRecords.stream().map(record -> record.getKey().getRecordKey()).distinct().count());
     assertEquals(numRecords, taggedRecords.stream().filter(record -> (record.getCurrentLocation() != null
@@ -210,8 +211,8 @@
   public void testSimpleTagLocationAndUpdateWithRollback() throws Exception {
     // Load to memory
     HoodieWriteConfig config = getConfig();
-    HBaseIndex index = new HBaseIndex(config);
-    HoodieWriteClient writeClient = getHoodieWriteClient(config);
+    SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config);
+    SparkRDDWriteClient writeClient = getHoodieWriteClient(config);
 
     final String newCommitTime = writeClient.startCommit();
     final int numRecords = 10;
@@ -225,9 +226,9 @@
 
     // commit this upsert
     writeClient.commit(newCommitTime, writeStatues);
-    HoodieTable hoodieTable = HoodieTable.create(metaClient, config, hadoopConf);
+    HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
     // Now tagLocation for these records, hbaseIndex should tag them
-    List<HoodieRecord> records2 = index.tagLocation(writeRecords, jsc(), hoodieTable).collect();
+    List<HoodieRecord> records2 = index.tagLocation(writeRecords, context(), hoodieTable).collect();
     assertEquals(numRecords, records2.stream().filter(HoodieRecord::isCurrentLocationKnown).count());
 
     // check tagged records are tagged with correct fileIds
@@ -240,10 +241,10 @@
     // Rollback the last commit
     writeClient.rollback(newCommitTime);
 
-    hoodieTable = HoodieTable.create(metaClient, config, hadoopConf);
+    hoodieTable = HoodieSparkTable.create(config, context, metaClient);
     // Now tagLocation for these records, hbaseIndex should not tag them since it was a rolled
     // back commit
-    List<HoodieRecord> records3 = index.tagLocation(writeRecords, jsc(), hoodieTable).collect();
+    List<HoodieRecord> records3 = index.tagLocation(writeRecords, context(), hoodieTable).collect();
     assertEquals(0, records3.stream().filter(HoodieRecord::isCurrentLocationKnown).count());
     assertEquals(0, records3.stream().filter(record -> record.getCurrentLocation() != null).count());
   }
@@ -251,7 +252,7 @@
   @Test
   public void testTotalGetsBatching() throws Exception {
     HoodieWriteConfig config = getConfig();
-    HBaseIndex index = new HBaseIndex(config);
+    SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config);
 
     // Mock hbaseConnection and related entities
     Connection hbaseConnection = mock(Connection.class);
@@ -262,21 +263,21 @@
     // only for test, set the hbaseConnection to mocked object
     index.setHbaseConnection(hbaseConnection);
 
-    HoodieWriteClient writeClient = getHoodieWriteClient(config);
+    SparkRDDWriteClient writeClient = getHoodieWriteClient(config);
 
     // start a commit and generate test data
     String newCommitTime = writeClient.startCommit();
     List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 250);
     JavaRDD<HoodieRecord> writeRecords = jsc().parallelize(records, 1);
     metaClient = HoodieTableMetaClient.reload(metaClient);
-    HoodieTable hoodieTable = HoodieTable.create(metaClient, config, hadoopConf);
+    HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
 
     // Insert 250 records
     JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
     assertNoWriteErrors(writeStatues.collect());
 
     // Now tagLocation for these records, hbaseIndex should tag them
-    index.tagLocation(writeRecords, jsc(), hoodieTable);
+    index.tagLocation(writeRecords, context(), hoodieTable);
 
     // 3 batches should be executed given batchSize = 100 and parallelism = 1
     verify(table, times(3)).get((List<Get>) any());
@@ -286,15 +287,15 @@
   @Test
   public void testTotalPutsBatching() throws Exception {
     HoodieWriteConfig config = getConfig();
-    HBaseIndex index = new HBaseIndex(config);
-    HoodieWriteClient writeClient = getHoodieWriteClient(config);
+    SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config);
+    SparkRDDWriteClient writeClient = getHoodieWriteClient(config);
 
     // start a commit and generate test data
     String newCommitTime = writeClient.startCommit();
     List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 250);
     JavaRDD<HoodieRecord> writeRecords = jsc().parallelize(records, 1);
     metaClient = HoodieTableMetaClient.reload(metaClient);
-    HoodieTable hoodieTable = HoodieTable.create(metaClient, config, hadoopConf);
+    HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
 
     // Insert 200 records
     JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
@@ -314,7 +315,7 @@
     // Get all the files generated
     int numberOfDataFileIds = (int) writeStatues.map(status -> status.getFileId()).distinct().count();
 
-    index.updateLocation(writeStatues, jsc(), hoodieTable);
+    index.updateLocation(writeStatues, context(), hoodieTable);
     // 3 batches should be executed given batchSize = 100 and <=numberOfDataFileIds getting updated,
     // so each fileId ideally gets updates
     verify(table, atMost(numberOfDataFileIds)).put((List<Put>) any());
@@ -323,7 +324,7 @@
   @Test
   public void testsHBasePutAccessParallelism() {
     HoodieWriteConfig config = getConfig();
-    HBaseIndex index = new HBaseIndex(config);
+    SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config);
     final JavaRDD<WriteStatus> writeStatusRDD = jsc().parallelize(
         Arrays.asList(getSampleWriteStatus(1, 2), getSampleWriteStatus(0, 3), getSampleWriteStatus(10, 0)), 10);
     final Tuple2<Long, Integer> tuple = index.getHBasePutAccessParallelism(writeStatusRDD);
@@ -337,7 +338,7 @@
   @Test
   public void testsHBasePutAccessParallelismWithNoInserts() {
     HoodieWriteConfig config = getConfig();
-    HBaseIndex index = new HBaseIndex(config);
+    SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config);
     final JavaRDD<WriteStatus> writeStatusRDD =
         jsc().parallelize(Arrays.asList(getSampleWriteStatus(0, 2), getSampleWriteStatus(0, 1)), 10);
     final Tuple2<Long, Integer> tuple = index.getHBasePutAccessParallelism(writeStatusRDD);
@@ -357,13 +358,13 @@
 
     // Load to memory
     HoodieWriteConfig config = getConfig(2);
-    HBaseIndex index = new HBaseIndex(config);
-    try (HoodieWriteClient writeClient = getHoodieWriteClient(config);) {
+    SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config);
+    try (SparkRDDWriteClient writeClient = getHoodieWriteClient(config);) {
       metaClient = HoodieTableMetaClient.reload(metaClient);
-      HoodieTable hoodieTable = HoodieTable.create(metaClient, config, hadoopConf);
+      HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
 
       // Test tagLocation without any entries in index
-      JavaRDD<HoodieRecord> records1 = index.tagLocation(writeRecords, jsc(), hoodieTable);
+      JavaRDD<HoodieRecord> records1 = index.tagLocation(writeRecords, context(), hoodieTable);
       assertEquals(0, records1.filter(record -> record.isCurrentLocationKnown()).count());
       // Insert 200 records
       writeClient.startCommitWithTime(newCommitTime);
@@ -372,15 +373,15 @@
 
       // Now tagLocation for these records, hbaseIndex should not tag them since it was a failed
       // commit
-      JavaRDD<HoodieRecord> records2 = index.tagLocation(writeRecords, jsc(), hoodieTable);
+      JavaRDD<HoodieRecord> records2 = index.tagLocation(writeRecords, context(), hoodieTable);
       assertEquals(0, records2.filter(record -> record.isCurrentLocationKnown()).count());
 
       // Now commit this & update location of records inserted and validate no errors
       writeClient.commit(newCommitTime, writeStatues);
       // Now tagLocation for these records, hbaseIndex should tag them correctly
       metaClient = HoodieTableMetaClient.reload(metaClient);
-      hoodieTable = HoodieTable.create(metaClient, config, hadoopConf);
-      List<HoodieRecord> records3 = index.tagLocation(writeRecords, jsc(), hoodieTable).collect();
+      hoodieTable = HoodieSparkTable.create(config, context, metaClient);
+      List<HoodieRecord> records3 = index.tagLocation(writeRecords, context(), hoodieTable).collect();
       assertEquals(numRecords, records3.stream().filter(record -> record.isCurrentLocationKnown()).count());
       assertEquals(numRecords, records3.stream().map(record -> record.getKey().getRecordKey()).distinct().count());
       assertEquals(numRecords, records3.stream().filter(record -> (record.getCurrentLocation() != null
@@ -397,13 +398,13 @@
 
     // Load to memory
     HoodieWriteConfig config = getConfig();
-    HBaseIndex index = new HBaseIndex(config);
-    try (HoodieWriteClient writeClient = getHoodieWriteClient(config);) {
+    SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config);
+    try (SparkRDDWriteClient writeClient = getHoodieWriteClient(config);) {
       metaClient = HoodieTableMetaClient.reload(metaClient);
-      HoodieTable hoodieTable = HoodieTable.create(metaClient, config, hadoopConf);
+      HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
 
       // Test tagLocation without any entries in index
-      JavaRDD<HoodieRecord> records1 = index.tagLocation(writeRecords, jsc(), hoodieTable);
+      JavaRDD<HoodieRecord> records1 = index.tagLocation(writeRecords, context(), hoodieTable);
       assertEquals(0, records1.filter(record -> record.isCurrentLocationKnown()).count());
 
       // Insert records
@@ -414,8 +415,8 @@
 
       // Now tagLocation for these records, hbaseIndex should tag them correctly
       metaClient = HoodieTableMetaClient.reload(metaClient);
-      hoodieTable = HoodieTable.create(metaClient, config, hadoopConf);
-      List<HoodieRecord> records2 = index.tagLocation(writeRecords, jsc(), hoodieTable).collect();
+      hoodieTable = HoodieSparkTable.create(config, context, metaClient);
+      List<HoodieRecord> records2 = index.tagLocation(writeRecords, context(), hoodieTable).collect();
       assertEquals(numRecords, records2.stream().filter(record -> record.isCurrentLocationKnown()).count());
       assertEquals(numRecords, records2.stream().map(record -> record.getKey().getRecordKey()).distinct().count());
       assertEquals(numRecords, records2.stream().filter(record -> (record.getCurrentLocation() != null
@@ -430,12 +431,12 @@
         newWriteStatus.setStat(new HoodieWriteStat());
         return newWriteStatus;
       });
-      JavaRDD<WriteStatus> deleteStatus = index.updateLocation(deleteWriteStatues, jsc(), hoodieTable);
+      JavaRDD<WriteStatus> deleteStatus = index.updateLocation(deleteWriteStatues, context(), hoodieTable);
       assertEquals(deleteStatus.count(), deleteWriteStatues.count());
       assertNoWriteErrors(deleteStatus.collect());
 
       // Ensure no records can be tagged
-      List<HoodieRecord> records3 = index.tagLocation(writeRecords, jsc(), hoodieTable).collect();
+      List<HoodieRecord> records3 = index.tagLocation(writeRecords, context(), hoodieTable).collect();
       assertEquals(0, records3.stream().filter(record -> record.isCurrentLocationKnown()).count());
       assertEquals(numRecords, records3.stream().map(record -> record.getKey().getRecordKey()).distinct().count());
       assertEquals(0, records3.stream().filter(record -> (record.getCurrentLocation() != null
diff --git a/hudi-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseIndexUsage.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseIndexUsage.java
similarity index 82%
rename from hudi-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseIndexUsage.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseIndexUsage.java
index c1bf157..6411b52 100644
--- a/hudi-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseIndexUsage.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseIndexUsage.java
@@ -25,7 +25,6 @@
 import org.junit.jupiter.api.extension.ExtendWith;
 import org.mockito.junit.jupiter.MockitoExtension;
 
-import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.mockito.Mockito.mock;
 
@@ -35,11 +34,7 @@
   @Test
   public void testFeatureSupport() {
     HoodieWriteConfig config = mock(HoodieWriteConfig.class);
-    HBaseIndex index = new HBaseIndex(config);
-
+    SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config);
     assertTrue(index.canIndexLogFiles());
-    assertThrows(UnsupportedOperationException.class, () -> {
-      index.fetchRecordLocation(null, null, null);
-    }, "HBaseIndex should not support fetchRecordLocation");
   }
 }
diff --git a/hudi-client/src/test/java/org/apache/hudi/index/hbase/TestHBasePutBatchSizeCalculator.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestHBasePutBatchSizeCalculator.java
similarity index 94%
rename from hudi-client/src/test/java/org/apache/hudi/index/hbase/TestHBasePutBatchSizeCalculator.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestHBasePutBatchSizeCalculator.java
index 3109942..e698eaf 100644
--- a/hudi-client/src/test/java/org/apache/hudi/index/hbase/TestHBasePutBatchSizeCalculator.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestHBasePutBatchSizeCalculator.java
@@ -19,8 +19,6 @@
 
 package org.apache.hudi.index.hbase;
 
-import org.apache.hudi.index.hbase.HBaseIndex.HBasePutBatchSizeCalculator;
-
 import org.junit.jupiter.api.Test;
 
 import static org.junit.jupiter.api.Assertions.assertEquals;
@@ -29,7 +27,7 @@
 
   @Test
   public void testPutBatchSizeCalculation() {
-    HBasePutBatchSizeCalculator batchSizeCalculator = new HBasePutBatchSizeCalculator();
+    SparkHoodieHBaseIndex.HBasePutBatchSizeCalculator batchSizeCalculator = new SparkHoodieHBaseIndex.HBasePutBatchSizeCalculator();
 
     // All asserts cases below are derived out of the first
     // example below, with change in one parameter at a time.
diff --git a/hudi-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseQPSResourceAllocator.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseQPSResourceAllocator.java
similarity index 95%
rename from hudi-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseQPSResourceAllocator.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseQPSResourceAllocator.java
index 8972d00..e9f8b87 100644
--- a/hudi-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseQPSResourceAllocator.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseQPSResourceAllocator.java
@@ -36,7 +36,7 @@
   @Test
   public void testsDefaultQPSResourceAllocator() {
     HoodieWriteConfig config = getConfig(Option.empty());
-    HBaseIndex index = new HBaseIndex(config);
+    SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config);
     HBaseIndexQPSResourceAllocator hBaseIndexQPSResourceAllocator = index.createQPSResourceAllocator(config);
     assertEquals(hBaseIndexQPSResourceAllocator.getClass().getName(),
         DefaultHBaseQPSResourceAllocator.class.getName());
@@ -47,7 +47,7 @@
   @Test
   public void testsExplicitDefaultQPSResourceAllocator() {
     HoodieWriteConfig config = getConfig(Option.of(HoodieHBaseIndexConfig.DEFAULT_HBASE_INDEX_QPS_ALLOCATOR_CLASS));
-    HBaseIndex index = new HBaseIndex(config);
+    SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config);
     HBaseIndexQPSResourceAllocator hBaseIndexQPSResourceAllocator = index.createQPSResourceAllocator(config);
     assertEquals(hBaseIndexQPSResourceAllocator.getClass().getName(),
         DefaultHBaseQPSResourceAllocator.class.getName());
@@ -58,7 +58,7 @@
   @Test
   public void testsInvalidQPSResourceAllocator() {
     HoodieWriteConfig config = getConfig(Option.of("InvalidResourceAllocatorClassName"));
-    HBaseIndex index = new HBaseIndex(config);
+    SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config);
     HBaseIndexQPSResourceAllocator hBaseIndexQPSResourceAllocator = index.createQPSResourceAllocator(config);
     assertEquals(hBaseIndexQPSResourceAllocator.getClass().getName(),
         DefaultHBaseQPSResourceAllocator.class.getName());
diff --git a/hudi-client/src/test/java/org/apache/hudi/io/TestHoodieKeyLocationFetchHandle.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieKeyLocationFetchHandle.java
similarity index 95%
rename from hudi-client/src/test/java/org/apache/hudi/io/TestHoodieKeyLocationFetchHandle.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieKeyLocationFetchHandle.java
index a7cd503..38cd19c 100644
--- a/hudi-client/src/test/java/org/apache/hudi/io/TestHoodieKeyLocationFetchHandle.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieKeyLocationFetchHandle.java
@@ -18,6 +18,7 @@
 
 package org.apache.hudi.io;
 
+import org.apache.hudi.client.common.HoodieEngineContext;
 import org.apache.hudi.common.fs.ConsistencyGuardConfig;
 import org.apache.hudi.common.model.HoodieBaseFile;
 import org.apache.hudi.common.model.HoodieKey;
@@ -31,12 +32,12 @@
 import org.apache.hudi.config.HoodieStorageConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.index.HoodieIndexUtils;
+import org.apache.hudi.table.HoodieSparkTable;
 import org.apache.hudi.table.HoodieTable;
 import org.apache.hudi.testutils.HoodieClientTestHarness;
 import org.apache.hudi.testutils.HoodieWriteableTestTable;
 import org.apache.hudi.testutils.MetadataMergeWriteStatus;
 
-import org.apache.spark.api.java.JavaSparkContext;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
@@ -85,12 +86,12 @@
   public void testFetchHandle() throws Exception {
     List<HoodieRecord> records = dataGen.generateInserts(makeNewCommitTime(), 100);
     Map<String, List<HoodieRecord>> partitionRecordsMap = recordsToPartitionRecordsMap(records);
-    HoodieTable hoodieTable = HoodieTable.create(metaClient, config, jsc.hadoopConfiguration());
+    HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
     HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, AVRO_SCHEMA_WITH_METADATA_FIELDS);
     Map<Tuple2<String, String>, List<Tuple2<HoodieKey, HoodieRecordLocation>>> expectedList =
         writeToParquetAndGetExpectedRecordLocations(partitionRecordsMap, testTable);
 
-    List<Tuple2<String, HoodieBaseFile>> partitionPathFileIdPairs = loadAllFilesForPartitions(new ArrayList<>(partitionRecordsMap.keySet()), jsc, hoodieTable);
+    List<Tuple2<String, HoodieBaseFile>> partitionPathFileIdPairs = loadAllFilesForPartitions(new ArrayList<>(partitionRecordsMap.keySet()), context, hoodieTable);
 
     for (Tuple2<String, HoodieBaseFile> entry : partitionPathFileIdPairs) {
       HoodieKeyLocationFetchHandle fetcherHandle = new HoodieKeyLocationFetchHandle(config, hoodieTable, Pair.of(entry._1, entry._2));
@@ -141,10 +142,10 @@
     return expectedList;
   }
 
-  private static List<Tuple2<String, HoodieBaseFile>> loadAllFilesForPartitions(List<String> partitions, JavaSparkContext jsc,
+  private static List<Tuple2<String, HoodieBaseFile>> loadAllFilesForPartitions(List<String> partitions, HoodieEngineContext context,
       HoodieTable hoodieTable) {
     // Obtain the latest data files from all the partitions.
-    List<Pair<String, HoodieBaseFile>> partitionPathFileIDList = HoodieIndexUtils.getLatestBaseFilesForAllPartitions(partitions, jsc, hoodieTable);
+    List<Pair<String, HoodieBaseFile>> partitionPathFileIDList = HoodieIndexUtils.getLatestBaseFilesForAllPartitions(partitions, context, hoodieTable);
     return partitionPathFileIDList.stream()
         .map(pf -> new Tuple2<>(pf.getKey(), pf.getValue())).collect(toList());
   }
diff --git a/hudi-client/src/test/java/org/apache/hudi/io/TestHoodieMergeHandle.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieMergeHandle.java
similarity index 98%
rename from hudi-client/src/test/java/org/apache/hudi/io/TestHoodieMergeHandle.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieMergeHandle.java
index 17ca6c9..95ed61a 100644
--- a/hudi-client/src/test/java/org/apache/hudi/io/TestHoodieMergeHandle.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieMergeHandle.java
@@ -18,7 +18,7 @@
 
 package org.apache.hudi.io;
 
-import org.apache.hudi.client.HoodieWriteClient;
+import org.apache.hudi.client.SparkRDDWriteClient;
 import org.apache.hudi.client.WriteStatus;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.HoodieRecord;
@@ -77,7 +77,7 @@
 
     // Build a write config with bulkinsertparallelism set
     HoodieWriteConfig cfg = getConfigBuilder().build();
-    try (HoodieWriteClient client = getHoodieWriteClient(cfg);) {
+    try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) {
       FileSystem fs = FSUtils.getFs(basePath, hadoopConf);
 
       /**
@@ -219,7 +219,7 @@
   public void testHoodieMergeHandleWriteStatMetrics() throws Exception {
     // insert 100 records
     HoodieWriteConfig config = getConfigBuilder().build();
-    try (HoodieWriteClient writeClient = getHoodieWriteClient(config);) {
+    try (SparkRDDWriteClient writeClient = getHoodieWriteClient(config);) {
       String newCommitTime = "100";
       writeClient.startCommitWithTime(newCommitTime);
 
diff --git a/hudi-client/src/test/java/org/apache/hudi/io/TestHoodieRowCreateHandle.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieRowCreateHandle.java
similarity index 83%
rename from hudi-client/src/test/java/org/apache/hudi/io/TestHoodieRowCreateHandle.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieRowCreateHandle.java
index 93a0253..c7a313a 100644
--- a/hudi-client/src/test/java/org/apache/hudi/io/TestHoodieRowCreateHandle.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieRowCreateHandle.java
@@ -24,9 +24,11 @@
 import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.exception.HoodieInsertException;
+import org.apache.hudi.table.HoodieSparkTable;
 import org.apache.hudi.table.HoodieTable;
 import org.apache.hudi.testutils.HoodieClientTestHarness;
 
+import org.apache.hudi.testutils.SparkDatasetTestUtils;
 import org.apache.spark.sql.Dataset;
 import org.apache.spark.sql.Row;
 import org.apache.spark.sql.catalyst.InternalRow;
@@ -40,12 +42,6 @@
 import java.util.Random;
 import java.util.UUID;
 
-import static org.apache.hudi.testutils.SparkDatasetTestUtils.ENCODER;
-import static org.apache.hudi.testutils.SparkDatasetTestUtils.STRUCT_TYPE;
-import static org.apache.hudi.testutils.SparkDatasetTestUtils.getConfigBuilder;
-import static org.apache.hudi.testutils.SparkDatasetTestUtils.getInternalRowWithError;
-import static org.apache.hudi.testutils.SparkDatasetTestUtils.getRandomRows;
-import static org.apache.hudi.testutils.SparkDatasetTestUtils.toInternalRows;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
@@ -56,6 +52,7 @@
 /**
  * Unit tests {@link HoodieRowCreateHandle}.
  */
+@SuppressWarnings("checkstyle:LineLength")
 public class TestHoodieRowCreateHandle extends HoodieClientTestHarness {
 
   private static final Random RANDOM = new Random();
@@ -77,8 +74,8 @@
   @Test
   public void testRowCreateHandle() throws IOException {
     // init config and table
-    HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
-    HoodieTable table = HoodieTable.create(metaClient, cfg, hadoopConf);
+    HoodieWriteConfig cfg = SparkDatasetTestUtils.getConfigBuilder(basePath).build();
+    HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
     List<String> fileNames = new ArrayList<>();
     List<String> fileAbsPaths = new ArrayList<>();
 
@@ -91,10 +88,10 @@
       String fileId = UUID.randomUUID().toString();
       String instantTime = "000";
 
-      HoodieRowCreateHandle handle = new HoodieRowCreateHandle(table, cfg, partitionPath, fileId, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), STRUCT_TYPE);
+      HoodieRowCreateHandle handle = new HoodieRowCreateHandle(table, cfg, partitionPath, fileId, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), SparkDatasetTestUtils.STRUCT_TYPE);
       int size = 10 + RANDOM.nextInt(1000);
       // Generate inputs
-      Dataset<Row> inputRows = getRandomRows(sqlContext, size, partitionPath, false);
+      Dataset<Row> inputRows = SparkDatasetTestUtils.getRandomRows(sqlContext, size, partitionPath, false);
       if (totalInputRows == null) {
         totalInputRows = inputRows;
       } else {
@@ -118,29 +115,29 @@
   @Test
   public void testGlobalFailure() throws IOException {
     // init config and table
-    HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
-    HoodieTable table = HoodieTable.create(metaClient, cfg, hadoopConf);
+    HoodieWriteConfig cfg = SparkDatasetTestUtils.getConfigBuilder(basePath).build();
+    HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
     String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[0];
 
     // init some args
     String fileId = UUID.randomUUID().toString();
     String instantTime = "000";
 
-    HoodieRowCreateHandle handle = new HoodieRowCreateHandle(table, cfg, partitionPath, fileId, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), STRUCT_TYPE);
+    HoodieRowCreateHandle handle = new HoodieRowCreateHandle(table, cfg, partitionPath, fileId, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), SparkDatasetTestUtils.STRUCT_TYPE);
     int size = 10 + RANDOM.nextInt(1000);
     int totalFailures = 5;
     // Generate first batch of valid rows
-    Dataset<Row> inputRows = getRandomRows(sqlContext, size / 2, partitionPath, false);
-    List<InternalRow> internalRows = toInternalRows(inputRows, ENCODER);
+    Dataset<Row> inputRows = SparkDatasetTestUtils.getRandomRows(sqlContext, size / 2, partitionPath, false);
+    List<InternalRow> internalRows = SparkDatasetTestUtils.toInternalRows(inputRows, SparkDatasetTestUtils.ENCODER);
 
     // generate some failures rows
     for (int i = 0; i < totalFailures; i++) {
-      internalRows.add(getInternalRowWithError(partitionPath));
+      internalRows.add(SparkDatasetTestUtils.getInternalRowWithError(partitionPath));
     }
 
     // generate 2nd batch of valid rows
-    Dataset<Row> inputRows2 = getRandomRows(sqlContext, size / 2, partitionPath, false);
-    internalRows.addAll(toInternalRows(inputRows2, ENCODER));
+    Dataset<Row> inputRows2 = SparkDatasetTestUtils.getRandomRows(sqlContext, size / 2, partitionPath, false);
+    internalRows.addAll(SparkDatasetTestUtils.toInternalRows(inputRows2, SparkDatasetTestUtils.ENCODER));
 
     // issue writes
     try {
@@ -171,11 +168,11 @@
   @Test
   public void testInstantiationFailure() throws IOException {
     // init config and table
-    HoodieWriteConfig cfg = getConfigBuilder(basePath).withPath("/dummypath/abc/").build();
-    HoodieTable table = HoodieTable.create(metaClient, cfg, hadoopConf);
+    HoodieWriteConfig cfg = SparkDatasetTestUtils.getConfigBuilder(basePath).withPath("/dummypath/abc/").build();
+    HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
 
     try {
-      new HoodieRowCreateHandle(table, cfg, " def", UUID.randomUUID().toString(), "001", RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), STRUCT_TYPE);
+      new HoodieRowCreateHandle(table, cfg, " def", UUID.randomUUID().toString(), "001", RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), SparkDatasetTestUtils.STRUCT_TYPE);
       fail("Should have thrown exception");
     } catch (HoodieInsertException ioe) {
       // expected
@@ -183,7 +180,7 @@
   }
 
   private HoodieInternalWriteStatus writeAndGetWriteStatus(Dataset<Row> inputRows, HoodieRowCreateHandle handle) throws IOException {
-    List<InternalRow> internalRows = toInternalRows(inputRows, ENCODER);
+    List<InternalRow> internalRows = SparkDatasetTestUtils.toInternalRows(inputRows, SparkDatasetTestUtils.ENCODER);
     // issue writes
     for (InternalRow internalRow : internalRows) {
       handle.write(internalRow);
diff --git a/hudi-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiveLog.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiveLog.java
similarity index 94%
rename from hudi-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiveLog.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiveLog.java
index 46e117a..88f755a 100644
--- a/hudi-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiveLog.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiveLog.java
@@ -37,6 +37,8 @@
 import org.apache.hudi.common.testutils.HoodieTestUtils;
 import org.apache.hudi.config.HoodieCompactionConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.table.HoodieSparkTable;
+import org.apache.hudi.table.HoodieTable;
 import org.apache.hudi.table.HoodieTimelineArchiveLog;
 import org.apache.hudi.testutils.HoodieClientTestHarness;
 import org.junit.jupiter.api.AfterEach;
@@ -65,10 +67,11 @@
     initPath();
     initSparkContexts();
     initMetaClient();
-    hadoopConf = metaClient.getHadoopConf();
+    hadoopConf = context.getHadoopConf().get();
     metaClient.getFs().mkdirs(new Path(basePath));
     metaClient = HoodieTestUtils.init(hadoopConf, basePath);
     wrapperFs = metaClient.getFs();
+    hadoopConf.addResource(wrapperFs.getConf());
   }
 
   @AfterEach
@@ -82,8 +85,9 @@
         HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
             .withParallelism(2, 2).forTable("test-trip-table").build();
     metaClient = HoodieTableMetaClient.reload(metaClient);
-    HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, hadoopConf);
-    boolean result = archiveLog.archiveIfRequired(jsc);
+    HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
+    HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table);
+    boolean result = archiveLog.archiveIfRequired(context);
     assertTrue(result);
   }
 
@@ -160,8 +164,9 @@
     verifyInflightInstants(metaClient, 2);
 
     metaClient = HoodieTableMetaClient.reload(metaClient);
-    HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, hadoopConf);
-    assertTrue(archiveLog.archiveIfRequired(jsc));
+    HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
+    HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table);
+    assertTrue(archiveLog.archiveIfRequired(context));
 
     // reload the timeline and remove the remaining commits
     timeline = metaClient.getActiveTimeline().reload().getAllCommitsTimeline().filterCompletedInstants();
@@ -228,9 +233,10 @@
 
     metaClient = HoodieTableMetaClient.reload(metaClient);
     HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
+    HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
     assertEquals(4, timeline.countInstants(), "Loaded 4 commits and the count should match");
-    HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, hadoopConf);
-    boolean result = archiveLog.archiveIfRequired(jsc);
+    HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table);
+    boolean result = archiveLog.archiveIfRequired(context);
     assertTrue(result);
 
     FileStatus[] allFiles = metaClient.getFs().listStatus(new Path(basePath + "/" + HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH));
@@ -252,7 +258,8 @@
         .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 5).build())
         .build();
     metaClient = HoodieTableMetaClient.reload(metaClient);
-    HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, hadoopConf);
+    HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
+    HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table);
     // Requested Compaction
     HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
         new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "100"), wrapperFs.getConf());
@@ -284,7 +291,7 @@
 
     HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
     assertEquals(4, timeline.countInstants(), "Loaded 4 commits and the count should match");
-    boolean result = archiveLog.archiveIfRequired(jsc);
+    boolean result = archiveLog.archiveIfRequired(context);
     assertTrue(result);
     timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants();
     assertEquals(4, timeline.countInstants(), "Should not archive commits when maxCommitsToKeep is 5");
@@ -317,7 +324,8 @@
         .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 5).build())
         .build();
     metaClient = HoodieTableMetaClient.reload(metaClient);
-    HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, hadoopConf);
+    HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
+    HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table);
     HoodieTestDataGenerator.createCommitFile(basePath, "100", wrapperFs.getConf());
     HoodieTestDataGenerator.createCommitFile(basePath, "101", wrapperFs.getConf());
     HoodieTestDataGenerator.createCommitFile(basePath, "102", wrapperFs.getConf());
@@ -327,7 +335,7 @@
 
     HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
     assertEquals(6, timeline.countInstants(), "Loaded 6 commits and the count should match");
-    boolean result = archiveLog.archiveIfRequired(jsc);
+    boolean result = archiveLog.archiveIfRequired(context);
     assertTrue(result);
     timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants();
     assertTrue(timeline.containsOrBeforeTimelineStarts("100"), "Archived commits should always be safe");
@@ -342,6 +350,7 @@
         .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2).forTable("test-trip-table")
         .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 5).build())
         .build();
+
     HoodieTestDataGenerator.createCommitFile(basePath, "100", wrapperFs.getConf());
     HoodieTestDataGenerator.createCommitFile(basePath, "101", wrapperFs.getConf());
     HoodieTestDataGenerator.createSavepointFile(basePath, "101", wrapperFs.getConf());
@@ -349,11 +358,12 @@
     HoodieTestDataGenerator.createCommitFile(basePath, "103", wrapperFs.getConf());
     HoodieTestDataGenerator.createCommitFile(basePath, "104", wrapperFs.getConf());
     HoodieTestDataGenerator.createCommitFile(basePath, "105", wrapperFs.getConf());
-    HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, hadoopConf);
+    HoodieTable table = HoodieSparkTable.create(cfg, context);
+    HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table);
 
     HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
     assertEquals(6, timeline.countInstants(), "Loaded 6 commits and the count should match");
-    assertTrue(archiveLog.archiveIfRequired(jsc));
+    assertTrue(archiveLog.archiveIfRequired(context));
     timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants();
     assertEquals(5, timeline.countInstants(),
         "Since we have a savepoint at 101, we should never archive any commit after 101 (we only archive 100)");
@@ -383,11 +393,12 @@
     HoodieTestDataGenerator.createCommitFile(basePath, "105", wrapperFs.getConf());
     HoodieTestDataGenerator.createCommitFile(basePath, "106", wrapperFs.getConf());
     HoodieTestDataGenerator.createCommitFile(basePath, "107", wrapperFs.getConf());
-    HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, wrapperFs.getConf());
+    HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
+    HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table);
 
     HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsAndCompactionTimeline();
     assertEquals(8, timeline.countInstants(), "Loaded 6 commits and the count should match");
-    boolean result = archiveLog.archiveIfRequired(jsc);
+    boolean result = archiveLog.archiveIfRequired(context);
     assertTrue(result);
     timeline = metaClient.getActiveTimeline().reload().getCommitsAndCompactionTimeline();
     assertFalse(timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "100")),
@@ -433,9 +444,9 @@
     HoodieTestDataGenerator.createCommitFile(basePath, "4", wrapperFs.getConf());
     HoodieTestDataGenerator.createCommitFile(basePath, "5", wrapperFs.getConf());
 
-
-    HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, wrapperFs.getConf());
-    boolean result = archiveLog.archiveIfRequired(jsc);
+    HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
+    HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table);
+    boolean result = archiveLog.archiveIfRequired(context);
     assertTrue(result);
     HoodieArchivedTimeline archivedTimeline = metaClient.getArchivedTimeline();
     List<HoodieInstant> archivedInstants = Arrays.asList(instant1, instant2, instant3);
@@ -460,7 +471,8 @@
         .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 5).build())
         .build();
     metaClient = HoodieTableMetaClient.reload(metaClient);
-    HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, hadoopConf);
+    HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
+    HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table);
 
     org.apache.hudi.avro.model.HoodieCommitMetadata expectedCommitMetadata = archiveLog.convertCommitMetadata(hoodieCommitMetadata);
     assertEquals(expectedCommitMetadata.getOperationType(), WriteOperationType.INSERT.toString());
diff --git a/hudi-client/src/test/java/org/apache/hudi/client/utils/TestSparkConfigUtils.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestSparkIOUtils.java
similarity index 73%
rename from hudi-client/src/test/java/org/apache/hudi/client/utils/TestSparkConfigUtils.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestSparkIOUtils.java
index 9463bfb..ffbf6d1 100644
--- a/hudi-client/src/test/java/org/apache/hudi/client/utils/TestSparkConfigUtils.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestSparkIOUtils.java
@@ -16,10 +16,12 @@
  * limitations under the License.
  */
 
-package org.apache.hudi.client.utils;
+package org.apache.hudi.io;
 
+import org.apache.hudi.client.SparkTaskContextSupplier;
 import org.apache.hudi.config.HoodieMemoryConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
+
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.io.TempDir;
 
@@ -29,10 +31,12 @@
 import static org.apache.hudi.config.HoodieMemoryConfig.MAX_MEMORY_FRACTION_FOR_MERGE_PROP;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 
-public class TestSparkConfigUtils {
+public class TestSparkIOUtils {
   @TempDir
   public java.nio.file.Path basePath;
 
+  private final SparkTaskContextSupplier contextSupplier = new SparkTaskContextSupplier();
+
   @Test
   public void testMaxMemoryPerPartitionMergeWithMaxSizeDefined() {
     String path = basePath.toString();
@@ -43,8 +47,8 @@
     HoodieMemoryConfig memoryConfig = HoodieMemoryConfig.newBuilder().withMaxMemoryMaxSize(mergeMaxSize, compactionMaxSize).build();
     HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(path).withMemoryConfig(memoryConfig).build();
 
-    assertEquals(mergeMaxSize, SparkConfigUtils.getMaxMemoryPerPartitionMerge(config.getProps()));
-    assertEquals(compactionMaxSize, SparkConfigUtils.getMaxMemoryPerCompaction(config.getProps()));
+    assertEquals(mergeMaxSize, IOUtils.getMaxMemoryPerPartitionMerge(contextSupplier, config.getProps()));
+    assertEquals(compactionMaxSize, IOUtils.getMaxMemoryPerCompaction(contextSupplier, config.getProps()));
   }
 
   @Test
@@ -54,12 +58,12 @@
     HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(path).build();
 
     String compactionFraction = config.getProps().getProperty(MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP, DEFAULT_MAX_MEMORY_FRACTION_FOR_COMPACTION);
-    long compactionMaxSize = SparkConfigUtils.getMaxMemoryAllowedForMerge(compactionFraction);
+    long compactionMaxSize = IOUtils.getMaxMemoryAllowedForMerge(contextSupplier, compactionFraction);
 
     String mergeFraction = config.getProps().getProperty(MAX_MEMORY_FRACTION_FOR_MERGE_PROP, DEFAULT_MAX_MEMORY_FRACTION_FOR_MERGE);
-    long mergeMaxSize = SparkConfigUtils.getMaxMemoryAllowedForMerge(mergeFraction);
+    long mergeMaxSize = IOUtils.getMaxMemoryAllowedForMerge(contextSupplier, mergeFraction);
 
-    assertEquals(mergeMaxSize, SparkConfigUtils.getMaxMemoryPerPartitionMerge(config.getProps()));
-    assertEquals(compactionMaxSize, SparkConfigUtils.getMaxMemoryPerCompaction(config.getProps()));
+    assertEquals(mergeMaxSize, IOUtils.getMaxMemoryPerPartitionMerge(contextSupplier, config.getProps()));
+    assertEquals(compactionMaxSize, IOUtils.getMaxMemoryPerCompaction(contextSupplier, config.getProps()));
   }
 }
diff --git a/hudi-client/src/test/java/org/apache/hudi/io/storage/TestHoodieFileWriterFactory.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/TestHoodieFileWriterFactory.java
similarity index 95%
rename from hudi-client/src/test/java/org/apache/hudi/io/storage/TestHoodieFileWriterFactory.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/TestHoodieFileWriterFactory.java
index 6ea2b7e..26f431a 100644
--- a/hudi-client/src/test/java/org/apache/hudi/io/storage/TestHoodieFileWriterFactory.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/TestHoodieFileWriterFactory.java
@@ -21,6 +21,7 @@
 import org.apache.hudi.client.SparkTaskContextSupplier;
 import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
 import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.table.HoodieSparkTable;
 import org.apache.hudi.table.HoodieTable;
 import org.apache.hudi.testutils.HoodieClientTestBase;
 
@@ -44,7 +45,7 @@
     final String instantTime = "100";
     final Path parquetPath = new Path(basePath + "/partition/path/f1_1-0-1_000.parquet");
     final HoodieWriteConfig cfg = getConfig();
-    HoodieTable table = HoodieTable.create(metaClient, cfg, hadoopConf);
+    HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
     SparkTaskContextSupplier supplier = new SparkTaskContextSupplier();
     HoodieFileWriter<IndexedRecord> parquetWriter = HoodieFileWriterFactory.getFileWriter(instantTime,
         parquetPath, table, cfg, HoodieTestDataGenerator.AVRO_SCHEMA, supplier);
diff --git a/hudi-client/src/test/java/org/apache/hudi/io/storage/TestHoodieInternalRowParquetWriter.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/TestHoodieInternalRowParquetWriter.java
similarity index 84%
rename from hudi-client/src/test/java/org/apache/hudi/io/storage/TestHoodieInternalRowParquetWriter.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/TestHoodieInternalRowParquetWriter.java
index bcb5aa6..37b8cdc 100644
--- a/hudi-client/src/test/java/org/apache/hudi/io/storage/TestHoodieInternalRowParquetWriter.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/TestHoodieInternalRowParquetWriter.java
@@ -26,6 +26,7 @@
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.hudi.testutils.SparkDatasetTestUtils;
 import org.apache.parquet.hadoop.metadata.CompressionCodecName;
 import org.apache.spark.sql.Dataset;
 import org.apache.spark.sql.Row;
@@ -39,11 +40,6 @@
 import java.util.Random;
 import java.util.UUID;
 
-import static org.apache.hudi.testutils.SparkDatasetTestUtils.ENCODER;
-import static org.apache.hudi.testutils.SparkDatasetTestUtils.STRUCT_TYPE;
-import static org.apache.hudi.testutils.SparkDatasetTestUtils.getConfigBuilder;
-import static org.apache.hudi.testutils.SparkDatasetTestUtils.getRandomRows;
-import static org.apache.hudi.testutils.SparkDatasetTestUtils.toInternalRows;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 
 /**
@@ -69,7 +65,7 @@
 
   @Test
   public void endToEndTest() throws IOException {
-    HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
+    HoodieWriteConfig cfg = SparkDatasetTestUtils.getConfigBuilder(basePath).build();
     for (int i = 0; i < 5; i++) {
       // init write support and parquet config
       HoodieRowParquetWriteSupport writeSupport = getWriteSupport(cfg, hadoopConf);
@@ -89,8 +85,8 @@
       // generate input
       int size = 10 + RANDOM.nextInt(100);
       // Generate inputs
-      Dataset<Row> inputRows = getRandomRows(sqlContext, size, partitionPath, false);
-      List<InternalRow> internalRows = toInternalRows(inputRows, ENCODER);
+      Dataset<Row> inputRows = SparkDatasetTestUtils.getRandomRows(sqlContext, size, partitionPath, false);
+      List<InternalRow> internalRows = SparkDatasetTestUtils.toInternalRows(inputRows, SparkDatasetTestUtils.ENCODER);
 
       // issue writes
       for (InternalRow internalRow : internalRows) {
@@ -112,6 +108,6 @@
         writeConfig.getBloomFilterFPP(),
         writeConfig.getDynamicBloomFilterMaxNumEntries(),
         writeConfig.getBloomFilterType());
-    return new HoodieRowParquetWriteSupport(hadoopConf, STRUCT_TYPE, filter);
+    return new HoodieRowParquetWriteSupport(hadoopConf, SparkDatasetTestUtils.STRUCT_TYPE, filter);
   }
 }
diff --git a/hudi-client/src/test/java/org/apache/hudi/table/TestCleaner.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java
similarity index 95%
rename from hudi-client/src/test/java/org/apache/hudi/table/TestCleaner.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java
index a4c7b5b..730e1ef 100644
--- a/hudi-client/src/test/java/org/apache/hudi/table/TestCleaner.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java
@@ -24,7 +24,7 @@
 import org.apache.hudi.avro.model.HoodieCleanerPlan;
 import org.apache.hudi.avro.model.HoodieCompactionPlan;
 import org.apache.hudi.avro.model.HoodieFileStatus;
-import org.apache.hudi.client.HoodieWriteClient;
+import org.apache.hudi.client.SparkRDDWriteClient;
 import org.apache.hudi.client.WriteStatus;
 import org.apache.hudi.common.HoodieCleanStat;
 import org.apache.hudi.common.bootstrap.TestBootstrapIndex;
@@ -63,6 +63,7 @@
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.exception.HoodieIOException;
 import org.apache.hudi.index.HoodieIndex;
+import org.apache.hudi.index.SparkHoodieIndex;
 import org.apache.hudi.table.action.clean.CleanPlanner;
 import org.apache.hudi.testutils.HoodieClientTestBase;
 
@@ -124,9 +125,9 @@
    * @param insertFn Insertion API for testing
    * @throws Exception in case of error
    */
-  private void insertFirstBigBatchForClientCleanerTest(HoodieWriteConfig cfg, HoodieWriteClient client,
+  private void insertFirstBigBatchForClientCleanerTest(HoodieWriteConfig cfg, SparkRDDWriteClient client,
       Function2<List<HoodieRecord>, String, Integer> recordGenFunction,
-      Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieRecord>, String> insertFn,
+      Function3<JavaRDD<WriteStatus>, SparkRDDWriteClient, JavaRDD<HoodieRecord>, String> insertFn,
       HoodieCleaningPolicy cleaningPolicy) throws Exception {
 
     /*
@@ -147,14 +148,14 @@
     HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
     assertEquals(1, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants(), "Expecting a single commit.");
     // Should have 100 records in table (check using Index), all in locations marked at commit
-    HoodieTable table = HoodieTable.create(metaClient, client.getConfig(), hadoopConf);
+    HoodieTable table = HoodieSparkTable.create(client.getConfig(), context, metaClient);
 
     assertFalse(table.getCompletedCommitsTimeline().empty());
     // We no longer write empty cleaner plans when there is nothing to be cleaned.
     assertTrue(table.getCompletedCleanTimeline().empty());
 
-    HoodieIndex index = HoodieIndex.createIndex(cfg);
-    List<HoodieRecord> taggedRecords = index.tagLocation(jsc.parallelize(records, 1), jsc, table).collect();
+    HoodieIndex index = SparkHoodieIndex.createIndex(cfg);
+    List<HoodieRecord> taggedRecords = ((JavaRDD<HoodieRecord>)index.tagLocation(jsc.parallelize(records, 1), context, table)).collect();
     checkTaggedRecords(taggedRecords, newCommitTime);
   }
 
@@ -163,7 +164,7 @@
    */
   @Test
   public void testInsertAndCleanByVersions() throws Exception {
-    testInsertAndCleanByVersions(HoodieWriteClient::insert, HoodieWriteClient::upsert, false);
+    testInsertAndCleanByVersions(SparkRDDWriteClient::insert, SparkRDDWriteClient::upsert, false);
   }
 
   /**
@@ -171,7 +172,7 @@
    */
   @Test
   public void testInsertPreppedAndCleanByVersions() throws Exception {
-    testInsertAndCleanByVersions(HoodieWriteClient::insertPreppedRecords, HoodieWriteClient::upsertPreppedRecords,
+    testInsertAndCleanByVersions(SparkRDDWriteClient::insertPreppedRecords, SparkRDDWriteClient::upsertPreppedRecords,
         true);
   }
 
@@ -180,7 +181,7 @@
    */
   @Test
   public void testBulkInsertAndCleanByVersions() throws Exception {
-    testInsertAndCleanByVersions(HoodieWriteClient::bulkInsert, HoodieWriteClient::upsert, false);
+    testInsertAndCleanByVersions(SparkRDDWriteClient::bulkInsert, SparkRDDWriteClient::upsert, false);
   }
 
   /**
@@ -190,7 +191,7 @@
   public void testBulkInsertPreppedAndCleanByVersions() throws Exception {
     testInsertAndCleanByVersions(
         (client, recordRDD, instantTime) -> client.bulkInsertPreppedRecords(recordRDD, instantTime, Option.empty()),
-        HoodieWriteClient::upsertPreppedRecords, true);
+        SparkRDDWriteClient::upsertPreppedRecords, true);
   }
 
   /**
@@ -203,8 +204,8 @@
    * @throws Exception in case of errors
    */
   private void testInsertAndCleanByVersions(
-      Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieRecord>, String> insertFn,
-      Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieRecord>, String> upsertFn, boolean isPreppedAPI)
+      Function3<JavaRDD<WriteStatus>, SparkRDDWriteClient, JavaRDD<HoodieRecord>, String> insertFn,
+      Function3<JavaRDD<WriteStatus>, SparkRDDWriteClient, JavaRDD<HoodieRecord>, String> upsertFn, boolean isPreppedAPI)
       throws Exception {
     int maxVersions = 2; // keep upto 2 versions for each file
     HoodieWriteConfig cfg = getConfigBuilder()
@@ -213,7 +214,7 @@
         .withParallelism(1, 1).withBulkInsertParallelism(1).withFinalizeWriteParallelism(1).withDeleteParallelism(1)
         .withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build())
         .build();
-    try (HoodieWriteClient client = getHoodieWriteClient(cfg);) {
+    try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) {
 
       final Function2<List<HoodieRecord>, String, Integer> recordInsertGenWrappedFunction =
           generateWrapRecordsFn(isPreppedAPI, cfg, dataGen::generateInserts);
@@ -226,7 +227,7 @@
 
       Map<HoodieFileGroupId, FileSlice> compactionFileIdToLatestFileSlice = new HashMap<>();
       metaClient = HoodieTableMetaClient.reload(metaClient);
-      HoodieTable table = HoodieTable.create(metaClient, getConfig(), hadoopConf);
+      HoodieTable table = HoodieSparkTable.create(getConfig(), context, metaClient);
       for (String partitionPath : dataGen.getPartitionPaths()) {
         TableFileSystemView fsView = table.getFileSystemView();
         Option<Boolean> added = Option.fromJavaOptional(fsView.getAllFileGroups(partitionPath).findFirst().map(fg -> {
@@ -263,7 +264,7 @@
           assertNoWriteErrors(statuses);
 
           metaClient = HoodieTableMetaClient.reload(metaClient);
-          table = HoodieTable.create(metaClient, getConfig(), hadoopConf);
+          table = HoodieSparkTable.create(getConfig(), context, metaClient);
           HoodieTimeline timeline = table.getMetaClient().getCommitsTimeline();
 
           TableFileSystemView fsView = table.getFileSystemView();
@@ -325,7 +326,7 @@
    */
   @Test
   public void testInsertAndCleanByCommits() throws Exception {
-    testInsertAndCleanByCommits(HoodieWriteClient::insert, HoodieWriteClient::upsert, false);
+    testInsertAndCleanByCommits(SparkRDDWriteClient::insert, SparkRDDWriteClient::upsert, false);
   }
 
   /**
@@ -333,7 +334,7 @@
    */
   @Test
   public void testInsertPreppedAndCleanByCommits() throws Exception {
-    testInsertAndCleanByCommits(HoodieWriteClient::insertPreppedRecords, HoodieWriteClient::upsertPreppedRecords, true);
+    testInsertAndCleanByCommits(SparkRDDWriteClient::insertPreppedRecords, SparkRDDWriteClient::upsertPreppedRecords, true);
   }
 
   /**
@@ -343,7 +344,7 @@
   public void testBulkInsertPreppedAndCleanByCommits() throws Exception {
     testInsertAndCleanByCommits(
         (client, recordRDD, instantTime) -> client.bulkInsertPreppedRecords(recordRDD, instantTime, Option.empty()),
-        HoodieWriteClient::upsertPreppedRecords, true);
+        SparkRDDWriteClient::upsertPreppedRecords, true);
   }
 
   /**
@@ -351,7 +352,7 @@
    */
   @Test
   public void testBulkInsertAndCleanByCommits() throws Exception {
-    testInsertAndCleanByCommits(HoodieWriteClient::bulkInsert, HoodieWriteClient::upsert, false);
+    testInsertAndCleanByCommits(SparkRDDWriteClient::bulkInsert, SparkRDDWriteClient::upsert, false);
   }
 
   /**
@@ -364,8 +365,8 @@
    * @throws Exception in case of errors
    */
   private void testInsertAndCleanByCommits(
-      Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieRecord>, String> insertFn,
-      Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieRecord>, String> upsertFn, boolean isPreppedAPI)
+      Function3<JavaRDD<WriteStatus>, SparkRDDWriteClient, JavaRDD<HoodieRecord>, String> insertFn,
+      Function3<JavaRDD<WriteStatus>, SparkRDDWriteClient, JavaRDD<HoodieRecord>, String> upsertFn, boolean isPreppedAPI)
       throws Exception {
     int maxCommits = 3; // keep upto 3 commits from the past
     HoodieWriteConfig cfg = getConfigBuilder()
@@ -374,7 +375,7 @@
         .withParallelism(1, 1).withBulkInsertParallelism(1).withFinalizeWriteParallelism(1).withDeleteParallelism(1)
         .withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build())
         .build();
-    HoodieWriteClient client = getHoodieWriteClient(cfg);
+    SparkRDDWriteClient client = getHoodieWriteClient(cfg);
 
     final Function2<List<HoodieRecord>, String, Integer> recordInsertGenWrappedFunction =
         generateWrapRecordsFn(isPreppedAPI, cfg, dataGen::generateInserts);
@@ -396,7 +397,7 @@
         assertNoWriteErrors(statuses);
 
         metaClient = HoodieTableMetaClient.reload(metaClient);
-        HoodieTable table1 = HoodieTable.create(metaClient, cfg, hadoopConf);
+        HoodieTable table1 = HoodieSparkTable.create(cfg, context, metaClient);
         HoodieTimeline activeTimeline = table1.getCompletedCommitsTimeline();
         // NOTE: See CleanPlanner#getFilesToCleanKeepingLatestCommits. We explicitly keep one commit before earliest
         // commit
@@ -452,7 +453,7 @@
    * @param config HoodieWriteConfig
    */
   private List<HoodieCleanStat> runCleaner(HoodieWriteConfig config, boolean simulateRetryFailure, int firstCommitSequence) throws IOException {
-    HoodieWriteClient<?> writeClient = getHoodieWriteClient(config);
+    SparkRDDWriteClient<?> writeClient = getHoodieWriteClient(config);
     String cleanInstantTs = makeNewCommitTime(firstCommitSequence);
     HoodieCleanMetadata cleanMetadata1 = writeClient.clean(cleanInstantTs);
 
@@ -986,11 +987,11 @@
 
     HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
     metaClient = HoodieTableMetaClient.reload(metaClient);
-    HoodieTable table = HoodieTable.create(metaClient, config, hadoopConf);
+    HoodieTable table = HoodieSparkTable.create(config, context, metaClient);
     table.getActiveTimeline().transitionRequestedToInflight(
         new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "000"), Option.empty());
     metaClient.reloadActiveTimeline();
-    table.rollback(jsc, "001", new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "000"), true);
+    table.rollback(context, "001", new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "000"), true);
     final int numTempFilesAfter = testTable.listAllFilesInTempFolder().size();
     assertEquals(0, numTempFilesAfter, "All temp files are deleted.");
   }
@@ -1129,7 +1130,7 @@
         if (j == i && j <= maxNumFileIdsForCompaction) {
           expFileIdToPendingCompaction.put(fileId, compactionInstants[j]);
           metaClient = HoodieTableMetaClient.reload(metaClient);
-          HoodieTable table = HoodieTable.create(metaClient, config, hadoopConf);
+          HoodieTable table = HoodieSparkTable.create(config, context, metaClient);
           FileSlice slice =
               table.getSliceView().getLatestFileSlices(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH)
                   .filter(fs -> fs.getFileId().equals(fileId)).findFirst().get();
@@ -1171,7 +1172,7 @@
 
     // Test for safety
     final HoodieTableMetaClient newMetaClient = HoodieTableMetaClient.reload(metaClient);
-    final HoodieTable hoodieTable = HoodieTable.create(metaClient, config, hadoopConf);
+    final HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
 
     expFileIdToPendingCompaction.forEach((fileId, value) -> {
       String baseInstantForCompaction = fileIdToLatestInstantBeforeCompaction.get(fileId);
diff --git a/hudi-client/src/test/java/org/apache/hudi/table/TestConsistencyGuard.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestConsistencyGuard.java
similarity index 100%
rename from hudi-client/src/test/java/org/apache/hudi/table/TestConsistencyGuard.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestConsistencyGuard.java
diff --git a/hudi-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java
similarity index 94%
rename from hudi-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java
index f603051..e7db7ad 100644
--- a/hudi-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java
@@ -19,8 +19,9 @@
 package org.apache.hudi.table;
 
 import org.apache.hudi.client.HoodieReadClient;
-import org.apache.hudi.client.HoodieWriteClient;
+import org.apache.hudi.client.SparkRDDWriteClient;
 import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.model.FileSlice;
 import org.apache.hudi.common.model.HoodieBaseFile;
 import org.apache.hudi.common.model.HoodieCommitMetadata;
@@ -57,8 +58,8 @@
 import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils;
 import org.apache.hudi.index.HoodieIndex;
 import org.apache.hudi.index.HoodieIndex.IndexType;
-import org.apache.hudi.table.action.deltacommit.DeleteDeltaCommitActionExecutor;
-import org.apache.hudi.table.action.deltacommit.DeltaCommitActionExecutor;
+import org.apache.hudi.table.action.deltacommit.AbstractSparkDeltaCommitActionExecutor;
+import org.apache.hudi.table.action.deltacommit.SparkDeleteDeltaCommitActionExecutor;
 import org.apache.hudi.testutils.HoodieClientTestHarness;
 import org.apache.hudi.testutils.HoodieClientTestUtils;
 import org.apache.hudi.testutils.HoodieMergeOnReadTestUtils;
@@ -110,6 +111,8 @@
     initDFS();
     initSparkContexts("TestHoodieMergeOnReadTable");
     hadoopConf.addResource(dfs.getConf());
+    jsc.hadoopConfiguration().addResource(dfs.getConf());
+    context = new HoodieSparkEngineContext(jsc);
     initPath();
     dfs.mkdirs(new Path(basePath));
     metaClient = HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.MERGE_ON_READ, baseFileFormat);
@@ -133,7 +136,7 @@
   @Test
   public void testSimpleInsertAndUpdate() throws Exception {
     HoodieWriteConfig cfg = getConfig(true);
-    try (HoodieWriteClient client = getHoodieWriteClient(cfg);) {
+    try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) {
 
       /**
        * Write 1 (only inserts)
@@ -155,7 +158,7 @@
       String compactionCommitTime = client.scheduleCompaction(Option.empty()).get().toString();
       client.compact(compactionCommitTime);
 
-      HoodieTable hoodieTable = HoodieTable.create(metaClient, cfg, hadoopConf);
+      HoodieTable hoodieTable = HoodieSparkTable.create(cfg, context, metaClient);
       FileStatus[] allFiles = listAllDataFilesInPath(hoodieTable, cfg.getBasePath());
       tableView = getHoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles);
       HoodieTableFileSystemView roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles);
@@ -181,7 +184,7 @@
     init(HoodieFileFormat.HFILE);
 
     HoodieWriteConfig cfg = getConfig(true);
-    try (HoodieWriteClient client = getHoodieWriteClient(cfg);) {
+    try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) {
 
       /**
        * Write 1 (only inserts)
@@ -203,7 +206,7 @@
       String compactionCommitTime = client.scheduleCompaction(Option.empty()).get().toString();
       client.compact(compactionCommitTime);
 
-      HoodieTable hoodieTable = HoodieTable.create(metaClient, cfg, hadoopConf);
+      HoodieTable hoodieTable = HoodieSparkTable.create(cfg, context, metaClient);
       FileStatus[] allFiles = listAllDataFilesInPath(hoodieTable, cfg.getBasePath());
       tableView = getHoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles);
       HoodieTableFileSystemView roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles);
@@ -230,7 +233,7 @@
     String partitionPath = "2020/02/20"; // use only one partition for this test
     dataGen = new HoodieTestDataGenerator(new String[] { partitionPath });
     HoodieWriteConfig cfg = getConfig(true);
-    try (HoodieWriteClient client = getHoodieWriteClient(cfg);) {
+    try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) {
 
       /**
        * Write 1 (only inserts)
@@ -327,7 +330,7 @@
   @Test
   public void testMetadataAggregateFromWriteStatus() throws Exception {
     HoodieWriteConfig cfg = getConfigBuilder(false).withWriteStatusClass(MetadataMergeWriteStatus.class).build();
-    try (HoodieWriteClient client = getHoodieWriteClient(cfg);) {
+    try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) {
 
       String newCommitTime = "001";
       List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 200);
@@ -350,7 +353,7 @@
   @Test
   public void testSimpleInsertUpdateAndDelete() throws Exception {
     HoodieWriteConfig cfg = getConfig(true);
-    try (HoodieWriteClient client = getHoodieWriteClient(cfg);) {
+    try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) {
 
       /**
        * Write 1 (only inserts, written as parquet file)
@@ -365,7 +368,7 @@
       assertNoWriteErrors(statuses);
 
       metaClient = getHoodieMetaClient(hadoopConf, cfg.getBasePath());
-      HoodieTable hoodieTable = HoodieTable.create(metaClient, cfg, hadoopConf);
+      HoodieTable hoodieTable = HoodieSparkTable.create(cfg, context, metaClient);
 
       Option<HoodieInstant> deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant();
       assertTrue(deltaCommit.isPresent());
@@ -432,7 +435,7 @@
     HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.COPY_ON_WRITE);
 
     HoodieWriteConfig cfg = getConfig(false, rollbackUsingMarkers);
-    try (HoodieWriteClient client = getHoodieWriteClient(cfg);) {
+    try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) {
 
       /**
        * Write 1 (only inserts)
@@ -472,7 +475,7 @@
       client.rollback(newCommitTime);
 
       metaClient = HoodieTableMetaClient.reload(metaClient);
-      HoodieTable hoodieTable = HoodieTable.create(metaClient, cfg, hadoopConf);
+      HoodieTable hoodieTable = HoodieSparkTable.create(cfg, context, metaClient);
       FileStatus[] allFiles = listAllDataFilesInPath(hoodieTable, cfg.getBasePath());
       tableView = getHoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles);
 
@@ -494,7 +497,7 @@
   private void testRollbackWithDeltaAndCompactionCommit(Boolean rollbackUsingMarkers) throws Exception {
     HoodieWriteConfig cfg = getConfig(false, rollbackUsingMarkers);
 
-    try (HoodieWriteClient client = getHoodieWriteClient(cfg);) {
+    try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) {
 
       // Test delta commit rollback
       /**
@@ -512,7 +515,7 @@
       assertNoWriteErrors(statuses);
 
       metaClient = getHoodieMetaClient(hadoopConf, cfg.getBasePath());
-      HoodieTable hoodieTable = HoodieTable.create(metaClient, cfg, hadoopConf);
+      HoodieTable hoodieTable = HoodieSparkTable.create(cfg, context, metaClient);
 
       Option<HoodieInstant> deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant();
       assertTrue(deltaCommit.isPresent());
@@ -537,7 +540,7 @@
        */
       final String commitTime1 = "002";
       // WriteClient with custom config (disable small file handling)
-      try (HoodieWriteClient secondClient = getHoodieWriteClient(getHoodieWriteConfigWithSmallFileHandlingOff());) {
+      try (SparkRDDWriteClient secondClient = getHoodieWriteClient(getHoodieWriteConfigWithSmallFileHandlingOff());) {
         secondClient.startCommitWithTime(commitTime1);
 
         List<HoodieRecord> copyOfRecords = new ArrayList<>(records);
@@ -570,7 +573,7 @@
        * Write 3 (inserts + updates - testing successful delta commit)
        */
       final String commitTime2 = "002";
-      try (HoodieWriteClient thirdClient = getHoodieWriteClient(cfg);) {
+      try (SparkRDDWriteClient thirdClient = getHoodieWriteClient(cfg);) {
         thirdClient.startCommitWithTime(commitTime2);
 
         List<HoodieRecord> copyOfRecords = new ArrayList<>(records);
@@ -596,7 +599,7 @@
             .filter(file -> file.getPath().getName().contains(commitTime2)).count());
 
         metaClient = HoodieTableMetaClient.reload(metaClient);
-        hoodieTable = HoodieTable.create(metaClient, cfg, hadoopConf);
+        hoodieTable = HoodieSparkTable.create(cfg, context, metaClient);
         tableView = getHoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles);
         dataFiles = tableView.getLatestBaseFiles().map(HoodieBaseFile::getPath).collect(Collectors.toList());
         recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(hadoopConf, dataFiles, basePath);
@@ -652,7 +655,7 @@
   @Test
   public void testMultiRollbackWithDeltaAndCompactionCommit() throws Exception {
     HoodieWriteConfig cfg = getConfig(false);
-    try (final HoodieWriteClient client = getHoodieWriteClient(cfg);) {
+    try (final SparkRDDWriteClient client = getHoodieWriteClient(cfg);) {
       /**
        * Write 1 (only inserts)
        */
@@ -668,7 +671,7 @@
       assertNoWriteErrors(statuses);
 
       metaClient = getHoodieMetaClient(hadoopConf, cfg.getBasePath());
-      HoodieTable hoodieTable = HoodieTable.create(metaClient, cfg, hadoopConf);
+      HoodieTable hoodieTable = HoodieSparkTable.create(cfg, context, metaClient);
 
       Option<HoodieInstant> deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant();
       assertTrue(deltaCommit.isPresent());
@@ -692,7 +695,7 @@
        */
       newCommitTime = "002";
       // WriteClient with custom config (disable small file handling)
-      HoodieWriteClient nClient = getHoodieWriteClient(getHoodieWriteConfigWithSmallFileHandlingOff());
+      SparkRDDWriteClient nClient = getHoodieWriteClient(getHoodieWriteConfigWithSmallFileHandlingOff());
       nClient.startCommitWithTime(newCommitTime);
 
       List<HoodieRecord> copyOfRecords = new ArrayList<>(records);
@@ -753,7 +756,7 @@
 
       compactionInstantTime = "006";
       client.scheduleCompactionAtInstant(compactionInstantTime, Option.empty());
-      JavaRDD<WriteStatus> ws = client.compact(compactionInstantTime);
+      JavaRDD<WriteStatus> ws = (JavaRDD<WriteStatus>) client.compact(compactionInstantTime);
       client.commitCompaction(compactionInstantTime, ws, Option.empty());
 
       allFiles = listAllDataFilesInPath(hoodieTable, cfg.getBasePath());
@@ -815,7 +818,7 @@
   @Test
   public void testUpsertPartitioner() throws Exception {
     HoodieWriteConfig cfg = getConfig(true);
-    try (HoodieWriteClient client = getHoodieWriteClient(cfg);) {
+    try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) {
 
       /**
        * Write 1 (only inserts, written as parquet file)
@@ -830,7 +833,7 @@
       assertNoWriteErrors(statuses);
 
       metaClient = getHoodieMetaClient(hadoopConf, cfg.getBasePath());
-      HoodieTable hoodieTable = HoodieTable.create(metaClient, cfg, hadoopConf);
+      HoodieTable hoodieTable = HoodieSparkTable.create(cfg, context, metaClient);
 
       Option<HoodieInstant> deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant();
       assertTrue(deltaCommit.isPresent());
@@ -895,7 +898,7 @@
   public void testLogFileCountsAfterCompaction() throws Exception {
     // insert 100 records
     HoodieWriteConfig config = getConfig(true);
-    try (HoodieWriteClient writeClient = getHoodieWriteClient(config);) {
+    try (SparkRDDWriteClient writeClient = getHoodieWriteClient(config);) {
       String newCommitTime = "100";
       writeClient.startCommitWithTime(newCommitTime);
 
@@ -912,7 +915,7 @@
       List<HoodieRecord> updatedRecords = dataGen.generateUpdates(newCommitTime, records);
       JavaRDD<HoodieRecord> updatedRecordsRDD = jsc.parallelize(updatedRecords, 1);
 
-      HoodieReadClient readClient = new HoodieReadClient(jsc, config);
+      HoodieReadClient readClient = new HoodieReadClient(context, config);
       updatedRecords = readClient.tagLocation(updatedRecordsRDD).collect();
 
       // Write them to corresponding avro logfiles
@@ -921,7 +924,7 @@
 
       // Verify that all data file has one log file
       metaClient = HoodieTableMetaClient.reload(metaClient);
-      HoodieTable table = HoodieTable.create(metaClient, config, hadoopConf);
+      HoodieTable table = HoodieSparkTable.create(config, context, metaClient);
       // In writeRecordsToLogFiles, no commit files are getting added, so resetting file-system view state
       ((SyncableFileSystemView) (table.getSliceView())).reset();
 
@@ -941,11 +944,11 @@
 
       // Do a compaction
       String compactionInstantTime = writeClient.scheduleCompaction(Option.empty()).get().toString();
-      JavaRDD<WriteStatus> result = writeClient.compact(compactionInstantTime);
+      JavaRDD<WriteStatus> result = (JavaRDD<WriteStatus>) writeClient.compact(compactionInstantTime);
 
       // Verify that recently written compacted data file has no log file
       metaClient = HoodieTableMetaClient.reload(metaClient);
-      table = HoodieTable.create(metaClient, config, hadoopConf);
+      table = HoodieSparkTable.create(config, context, metaClient);
       HoodieActiveTimeline timeline = metaClient.getActiveTimeline();
 
       assertTrue(HoodieTimeline
@@ -969,7 +972,7 @@
     // insert 100 records
     // Setting IndexType to be InMemory to simulate Global Index nature
     HoodieWriteConfig config = getConfigBuilder(false, IndexType.INMEMORY).build();
-    try (HoodieWriteClient writeClient = getHoodieWriteClient(config);) {
+    try (SparkRDDWriteClient writeClient = getHoodieWriteClient(config);) {
       String newCommitTime = "100";
       writeClient.startCommitWithTime(newCommitTime);
 
@@ -979,7 +982,7 @@
       writeClient.commit(newCommitTime, statuses);
 
       HoodieTable table =
-          HoodieTable.create(getHoodieMetaClient(hadoopConf, basePath), config, hadoopConf);
+          HoodieSparkTable.create(config, context, getHoodieMetaClient(hadoopConf, basePath));
       SliceView tableRTFileSystemView = table.getSliceView();
 
       long numLogFiles = 0;
@@ -994,7 +997,7 @@
       assertTrue(numLogFiles > 0);
       // Do a compaction
       String instantTime = writeClient.scheduleCompaction(Option.empty()).get().toString();
-      statuses = writeClient.compact(instantTime);
+      statuses = (JavaRDD<WriteStatus>) writeClient.compact(instantTime);
       assertEquals(statuses.map(status -> status.getStat().getPath().contains("parquet")).count(), numLogFiles);
       assertEquals(statuses.count(), numLogFiles);
       writeClient.commitCompaction(instantTime, statuses, Option.empty());
@@ -1005,7 +1008,7 @@
     // insert 100 records
     // Setting IndexType to be InMemory to simulate Global Index nature
     HoodieWriteConfig config = getConfigBuilder(false, rollbackUsingMarkers, IndexType.INMEMORY).build();
-    try (HoodieWriteClient writeClient = getHoodieWriteClient(config)) {
+    try (SparkRDDWriteClient writeClient = getHoodieWriteClient(config)) {
       String newCommitTime = "100";
       writeClient.startCommitWithTime(newCommitTime);
 
@@ -1059,7 +1062,7 @@
 
       writeClient.rollback(newCommitTime);
       metaClient = HoodieTableMetaClient.reload(metaClient);
-      HoodieTable table = HoodieTable.create(config, hadoopConf);
+      HoodieTable table = HoodieSparkTable.create(config, context);
       SliceView tableRTFileSystemView = table.getSliceView();
 
       long numLogFiles = 0;
@@ -1102,7 +1105,7 @@
     // insert 100 records
     // Setting IndexType to be InMemory to simulate Global Index nature
     HoodieWriteConfig config = getConfigBuilder(false, rollbackUsingMarkers, IndexType.INMEMORY).build();
-    try (HoodieWriteClient writeClient = getHoodieWriteClient(config);) {
+    try (SparkRDDWriteClient writeClient = getHoodieWriteClient(config);) {
       String newCommitTime = "100";
       writeClient.startCommitWithTime(newCommitTime);
 
@@ -1113,7 +1116,7 @@
       // trigger an action
       statuses.collect();
 
-      HoodieTable table = HoodieTable.create(getHoodieMetaClient(hadoopConf, basePath), config, hadoopConf);
+      HoodieTable table = HoodieSparkTable.create(config, context, getHoodieMetaClient(hadoopConf, basePath));
       SliceView tableRTFileSystemView = table.getSliceView();
 
       long numLogFiles = 0;
@@ -1127,7 +1130,7 @@
       assertTrue(numLogFiles > 0);
       // Do a compaction
       newCommitTime = writeClient.scheduleCompaction(Option.empty()).get().toString();
-      statuses = writeClient.compact(newCommitTime);
+      statuses = (JavaRDD<WriteStatus>) writeClient.compact(newCommitTime);
       // Ensure all log files have been compacted into parquet files
       assertEquals(statuses.map(status -> status.getStat().getPath().contains("parquet")).count(), numLogFiles);
       assertEquals(statuses.count(), numLogFiles);
@@ -1136,7 +1139,7 @@
       table.getActiveTimeline().reload();
       writeClient.rollbackInflightCompaction(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, newCommitTime), table);
 
-      table = HoodieTable.create(getHoodieMetaClient(hadoopConf, basePath), config, hadoopConf);
+      table = HoodieSparkTable.create(config, context, getHoodieMetaClient(hadoopConf, basePath));
       tableRTFileSystemView = table.getSliceView();
       ((SyncableFileSystemView) tableRTFileSystemView).reset();
 
@@ -1165,9 +1168,9 @@
   public void testMetadataStatsOnCommit(Boolean rollbackUsingMarkers) throws Exception {
     HoodieWriteConfig cfg = getConfigBuilder(false, rollbackUsingMarkers, IndexType.INMEMORY)
         .withAutoCommit(false).build();
-    try (HoodieWriteClient client = getHoodieWriteClient(cfg);) {
+    try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) {
       metaClient = getHoodieMetaClient(hadoopConf, basePath);
-      HoodieTable table = HoodieTable.create(metaClient, cfg, hadoopConf);
+      HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
 
       // Create a commit without metadata stats in metadata to test backwards compatibility
       HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
@@ -1188,7 +1191,7 @@
       assertTrue(client.commit(instantTime, statuses), "Commit should succeed");
 
       // Read from commit file
-      table = HoodieTable.create(cfg, hadoopConf);
+      table = HoodieSparkTable.create(cfg, context);
       HoodieCommitMetadata metadata = HoodieCommitMetadata.fromBytes(
           table.getActiveTimeline().getInstantDetails(table.getActiveTimeline().getDeltaCommitTimeline().lastInstant().get()).get(),
           HoodieCommitMetadata.class);
@@ -1221,7 +1224,7 @@
       client.rollback(instantTime);
 
       // Read from commit file
-      table = HoodieTable.create(cfg, hadoopConf);
+      table = HoodieSparkTable.create(cfg, context);
       metadata = HoodieCommitMetadata.fromBytes(
           table.getActiveTimeline()
               .getInstantDetails(table.getActiveTimeline().getDeltaCommitTimeline().lastInstant().get()).get(),
@@ -1258,7 +1261,7 @@
   @Test
   public void testRollingStatsWithSmallFileHandling() throws Exception {
     HoodieWriteConfig cfg = getConfigBuilder(false, IndexType.INMEMORY).withAutoCommit(false).build();
-    try (HoodieWriteClient client = getHoodieWriteClient(cfg);) {
+    try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) {
       Map<String, Long> fileIdToInsertsMap = new HashMap<>();
       Map<String, Long> fileIdToUpsertsMap = new HashMap<>();
 
@@ -1272,7 +1275,7 @@
       assertTrue(client.commit(instantTime, statuses), "Commit should succeed");
 
       // Read from commit file
-      HoodieTable table = HoodieTable.create(cfg, hadoopConf);
+      HoodieTable table = HoodieSparkTable.create(cfg, context);
       HoodieCommitMetadata metadata = HoodieCommitMetadata.fromBytes(
           table.getActiveTimeline()
               .getInstantDetails(table.getActiveTimeline().getDeltaCommitTimeline().lastInstant().get()).get(),
@@ -1297,7 +1300,7 @@
       assertTrue(client.commit(instantTime, statuses), "Commit should succeed");
 
       // Read from commit file
-      table = HoodieTable.create(cfg, hadoopConf);
+      table = HoodieSparkTable.create(cfg, context);
       metadata = HoodieCommitMetadata.fromBytes(
           table.getActiveTimeline()
               .getInstantDetails(table.getActiveTimeline().getDeltaCommitTimeline().lastInstant().get()).get(),
@@ -1319,11 +1322,11 @@
       // Test small file handling after compaction
       instantTime = "002";
       client.scheduleCompactionAtInstant(instantTime, Option.of(metadata.getExtraMetadata()));
-      statuses = client.compact(instantTime);
+      statuses = (JavaRDD<WriteStatus>) client.compact(instantTime);
       client.commitCompaction(instantTime, statuses, Option.empty());
 
       // Read from commit file
-      table = HoodieTable.create(cfg, hadoopConf);
+      table = HoodieSparkTable.create(cfg, context);
       HoodieCommitMetadata metadata1 = HoodieCommitMetadata.fromBytes(
           table.getActiveTimeline()
               .getInstantDetails(table.getActiveTimeline().getCommitsTimeline().lastInstant().get()).get(),
@@ -1347,7 +1350,7 @@
       assertTrue(client.commit(instantTime, statuses), "Commit should succeed");
 
       // Read from commit file
-      table = HoodieTable.create(cfg, hadoopConf);
+      table = HoodieSparkTable.create(cfg, context);
       metadata = HoodieCommitMetadata.fromBytes(
           table.getActiveTimeline()
               .getInstantDetails(table.getActiveTimeline().getDeltaCommitTimeline().lastInstant().get()).get(),
@@ -1373,7 +1376,7 @@
   @Test
   public void testHandleUpdateWithMultiplePartitions() throws Exception {
     HoodieWriteConfig cfg = getConfig(true);
-    try (HoodieWriteClient client = getHoodieWriteClient(cfg);) {
+    try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) {
 
       /**
        * Write 1 (only inserts, written as parquet file)
@@ -1388,7 +1391,7 @@
       assertNoWriteErrors(statuses);
 
       metaClient = getHoodieMetaClient(hadoopConf, cfg.getBasePath());
-      HoodieMergeOnReadTable hoodieTable = (HoodieMergeOnReadTable) HoodieTable.create(metaClient, cfg, hadoopConf);
+      HoodieSparkMergeOnReadTable hoodieTable = (HoodieSparkMergeOnReadTable) HoodieSparkTable.create(cfg, context, metaClient);
 
       Option<HoodieInstant> deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant();
       assertTrue(deltaCommit.isPresent());
@@ -1432,9 +1435,9 @@
       JavaRDD<HoodieRecord> deleteRDD = jsc.parallelize(fewRecordsForDelete, 1);
 
       // initialize partitioner
-      DeltaCommitActionExecutor actionExecutor = new DeleteDeltaCommitActionExecutor(jsc, cfg, hoodieTable,
+      AbstractSparkDeltaCommitActionExecutor actionExecutor = new SparkDeleteDeltaCommitActionExecutor(context, cfg, hoodieTable,
           newDeleteTime, deleteRDD);
-      actionExecutor.getUpsertPartitioner(new WorkloadProfile(deleteRDD));
+      actionExecutor.getUpsertPartitioner(new WorkloadProfile(buildProfile(deleteRDD)));
       final List<List<WriteStatus>> deleteStatus = jsc.parallelize(Arrays.asList(1)).map(x -> {
         return actionExecutor.handleUpdate(partitionPath, fileId, fewRecordsForDelete.iterator());
       }).map(Transformations::flatten).collect();
@@ -1479,7 +1482,7 @@
         .withRollbackUsingMarkers(rollbackUsingMarkers);
   }
 
-  private FileStatus[] insertAndGetFilePaths(List<HoodieRecord> records, HoodieWriteClient client,
+  private FileStatus[] insertAndGetFilePaths(List<HoodieRecord> records, SparkRDDWriteClient client,
                                              HoodieWriteConfig cfg, String commitTime) throws IOException {
     JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
 
@@ -1487,7 +1490,7 @@
     assertNoWriteErrors(statuses);
 
     metaClient = getHoodieMetaClient(hadoopConf, cfg.getBasePath());
-    HoodieTable hoodieTable = HoodieTable.create(metaClient, cfg, hadoopConf);
+    HoodieTable hoodieTable = HoodieSparkTable.create(cfg, context, metaClient);
 
     Option<HoodieInstant> deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().lastInstant();
     assertTrue(deltaCommit.isPresent());
@@ -1509,7 +1512,7 @@
     return allFiles;
   }
 
-  private FileStatus[] updateAndGetFilePaths(List<HoodieRecord> records, HoodieWriteClient client,
+  private FileStatus[] updateAndGetFilePaths(List<HoodieRecord> records, SparkRDDWriteClient client,
                                              HoodieWriteConfig cfg, String commitTime) throws IOException {
     Map<HoodieKey, HoodieRecord> recordsMap = new HashMap<>();
     for (HoodieRecord rec : records) {
@@ -1529,7 +1532,7 @@
 
     Option<HoodieInstant> commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant();
     assertFalse(commit.isPresent());
-    HoodieTable hoodieTable = HoodieTable.create(metaClient, cfg, hadoopConf);
+    HoodieTable hoodieTable = HoodieSparkTable.create(cfg, context, metaClient);
     return listAllDataFilesInPath(hoodieTable, cfg.getBasePath());
   }
 
diff --git a/hudi-client/src/test/java/org/apache/hudi/table/TestMarkerFiles.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestMarkerFiles.java
similarity index 93%
rename from hudi-client/src/test/java/org/apache/hudi/table/TestMarkerFiles.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestMarkerFiles.java
index 55b7b50..b25427b 100644
--- a/hudi-client/src/test/java/org/apache/hudi/table/TestMarkerFiles.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestMarkerFiles.java
@@ -18,6 +18,7 @@
 
 package org.apache.hudi.table;
 
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.IOType;
 import org.apache.hudi.common.testutils.FileSystemTestUtils;
@@ -49,12 +50,14 @@
   private FileSystem fs;
   private Path markerFolderPath;
   private JavaSparkContext jsc;
+  private HoodieSparkEngineContext context;
 
   @BeforeEach
   public void setup() throws IOException {
     initPath();
     initMetaClient();
     this.jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest(TestMarkerFiles.class.getName()));
+    this.context = new HoodieSparkEngineContext(jsc);
     this.fs = FSUtils.getFs(metaClient.getBasePath(), metaClient.getHadoopConf());
     this.markerFolderPath =  new Path(metaClient.getMarkerFolderPath("000"));
     this.markerFiles = new MarkerFiles(fs, metaClient.getBasePath(), markerFolderPath.toString(), "000");
@@ -63,6 +66,7 @@
   @AfterEach
   public void cleanup() {
     jsc.stop();
+    context = null;
   }
 
   private void createSomeMarkerFiles() {
@@ -107,7 +111,7 @@
 
     // then
     assertTrue(markerFiles.doesMarkerDirExist());
-    assertTrue(markerFiles.deleteMarkerDir(jsc, 2));
+    assertTrue(markerFiles.deleteMarkerDir(context, 2));
     assertFalse(markerFiles.doesMarkerDirExist());
   }
 
@@ -115,7 +119,7 @@
   public void testDeletionWhenMarkerDirNotExists() throws IOException {
     // then
     assertFalse(markerFiles.doesMarkerDirExist());
-    assertFalse(markerFiles.deleteMarkerDir(jsc, 2));
+    assertFalse(markerFiles.deleteMarkerDir(context, 2));
   }
 
   @Test
@@ -130,7 +134,7 @@
     // then
     assertIterableEquals(CollectionUtils.createImmutableList(
         "2020/06/01/file1", "2020/06/03/file3"),
-        markerFiles.createdAndMergedDataPaths(jsc, 2).stream().sorted().collect(Collectors.toList())
+        markerFiles.createdAndMergedDataPaths(context, 2).stream().sorted().collect(Collectors.toList())
     );
   }
 
diff --git a/hudi-client/src/test/java/org/apache/hudi/table/action/bootstrap/TestBootstrapUtils.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/bootstrap/TestBootstrapUtils.java
similarity index 95%
rename from hudi-client/src/test/java/org/apache/hudi/table/action/bootstrap/TestBootstrapUtils.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/bootstrap/TestBootstrapUtils.java
index 64db6cf..b57e1c5 100644
--- a/hudi-client/src/test/java/org/apache/hudi/table/action/bootstrap/TestBootstrapUtils.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/bootstrap/TestBootstrapUtils.java
@@ -64,14 +64,14 @@
     });
 
     List<Pair<String, List<HoodieFileStatus>>> collected = BootstrapUtils.getAllLeafFoldersWithFiles(metaClient,
-            metaClient.getFs(), basePath, jsc);
+            metaClient.getFs(), basePath, context);
     assertEquals(3, collected.size());
     collected.stream().forEach(k -> {
       assertEquals(2, k.getRight().size());
     });
 
     // Simulate reading from un-partitioned dataset
-    collected = BootstrapUtils.getAllLeafFoldersWithFiles(metaClient, metaClient.getFs(), basePath + "/" + folders.get(0), jsc);
+    collected = BootstrapUtils.getAllLeafFoldersWithFiles(metaClient, metaClient.getFs(), basePath + "/" + folders.get(0), context);
     assertEquals(1, collected.size());
     collected.stream().forEach(k -> {
       assertEquals(2, k.getRight().size());
diff --git a/hudi-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java
similarity index 90%
rename from hudi-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java
index 89fd592..852f802 100644
--- a/hudi-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java
@@ -18,7 +18,7 @@
 
 package org.apache.hudi.table.action.commit;
 
-import org.apache.hudi.client.HoodieWriteClient;
+import org.apache.hudi.client.SparkRDDWriteClient;
 import org.apache.hudi.client.WriteStatus;
 import org.apache.hudi.common.bloom.BloomFilter;
 import org.apache.hudi.common.fs.FSUtils;
@@ -37,7 +37,8 @@
 import org.apache.hudi.hadoop.HoodieParquetInputFormat;
 import org.apache.hudi.hadoop.utils.HoodieHiveUtils;
 import org.apache.hudi.io.HoodieCreateHandle;
-import org.apache.hudi.table.HoodieCopyOnWriteTable;
+import org.apache.hudi.table.HoodieSparkCopyOnWriteTable;
+import org.apache.hudi.table.HoodieSparkTable;
 import org.apache.hudi.table.HoodieTable;
 import org.apache.hudi.testutils.HoodieClientTestBase;
 import org.apache.hudi.testutils.MetadataMergeWriteStatus;
@@ -90,7 +91,7 @@
     String instantTime = makeNewCommitTime();
     HoodieWriteConfig config = makeHoodieClientConfig();
     metaClient = HoodieTableMetaClient.reload(metaClient);
-    HoodieTable table = HoodieTable.create(metaClient, config, hadoopConf);
+    HoodieTable table = HoodieSparkTable.create(config, context, metaClient);
 
     Pair<Path, String> newPathWithWriteToken = jsc.parallelize(Arrays.asList(1)).map(x -> {
       HoodieRecord record = mock(HoodieRecord.class);
@@ -120,12 +121,12 @@
     // Prepare the AvroParquetIO
     HoodieWriteConfig config = makeHoodieClientConfig();
     String firstCommitTime = makeNewCommitTime();
-    HoodieWriteClient writeClient = getHoodieWriteClient(config);
+    SparkRDDWriteClient writeClient = getHoodieWriteClient(config);
     writeClient.startCommitWithTime(firstCommitTime);
     metaClient = HoodieTableMetaClient.reload(metaClient);
 
     String partitionPath = "2016/01/31";
-    HoodieCopyOnWriteTable table = (HoodieCopyOnWriteTable) HoodieTable.create(metaClient, config, hadoopConf);
+    HoodieSparkCopyOnWriteTable table = (HoodieSparkCopyOnWriteTable) HoodieSparkTable.create(config, context, metaClient);
 
     // Get some records belong to the same partition (2016/01/31)
     String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\","
@@ -146,7 +147,7 @@
     records.add(new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3));
 
     // Insert new records
-    final HoodieCopyOnWriteTable cowTable = table;
+    final HoodieSparkCopyOnWriteTable cowTable = table;
     writeClient.insert(jsc.parallelize(records, 1), firstCommitTime);
 
     FileStatus[] allFiles = getIncrementalFiles(partitionPath, "0", -1);
@@ -267,7 +268,7 @@
     String firstCommitTime = makeNewCommitTime();
     metaClient = HoodieTableMetaClient.reload(metaClient);
 
-    HoodieCopyOnWriteTable table = (HoodieCopyOnWriteTable) HoodieTable.create(metaClient, config, hadoopConf);
+    HoodieSparkCopyOnWriteTable table = (HoodieSparkCopyOnWriteTable) HoodieSparkTable.create(config, context, metaClient);
 
     // Get some records belong to the same partition (2016/01/31)
     String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\","
@@ -286,7 +287,7 @@
     records.add(new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3));
 
     // Insert new records
-    CommitActionExecutor actionExecutor = new InsertCommitActionExecutor(jsc, config, table,
+    BaseSparkCommitActionExecutor actionExecutor = new SparkInsertCommitActionExecutor(context, config, table,
         firstCommitTime, jsc.parallelize(records));
     List<WriteStatus> writeStatuses = jsc.parallelize(Arrays.asList(1)).map(x -> {
       return actionExecutor.handleInsert(FSUtils.createNewFileIdPfx(), records.iterator());
@@ -320,7 +321,7 @@
     HoodieWriteConfig config = makeHoodieClientConfig();
     String instantTime = makeNewCommitTime();
     metaClient = HoodieTableMetaClient.reload(metaClient);
-    HoodieCopyOnWriteTable table = (HoodieCopyOnWriteTable) HoodieTable.create(metaClient, config, hadoopConf);
+    HoodieSparkCopyOnWriteTable table = (HoodieSparkCopyOnWriteTable) HoodieSparkTable.create(config, context, metaClient);
 
     // Case 1:
     // 10 records for partition 1, 1 record for partition 2.
@@ -329,7 +330,7 @@
 
     // Insert new records
     final List<HoodieRecord> recs2 = records;
-    CommitActionExecutor actionExecutor = new InsertPreppedCommitActionExecutor(jsc, config, table,
+    BaseSparkCommitActionExecutor actionExecutor = new SparkInsertPreppedCommitActionExecutor(context, config, table,
         instantTime, jsc.parallelize(recs2));
     List<WriteStatus> returnedStatuses = jsc.parallelize(Arrays.asList(1)).map(x -> {
       return actionExecutor.handleInsert(FSUtils.createNewFileIdPfx(), recs2.iterator());
@@ -350,7 +351,7 @@
 
     // Insert new records
     final List<HoodieRecord> recs3 = records;
-    CommitActionExecutor newActionExecutor = new UpsertPreppedCommitActionExecutor(jsc, config, table,
+    BaseSparkCommitActionExecutor newActionExecutor = new SparkUpsertPreppedCommitActionExecutor(context, config, table,
         instantTime, jsc.parallelize(recs3));
     returnedStatuses = jsc.parallelize(Arrays.asList(1)).map(x -> {
       return newActionExecutor.handleInsert(FSUtils.createNewFileIdPfx(), recs3.iterator());
@@ -371,7 +372,7 @@
         .parquetBlockSize(64 * 1024).parquetPageSize(64 * 1024).build()).build();
     String instantTime = makeNewCommitTime();
     metaClient = HoodieTableMetaClient.reload(metaClient);
-    HoodieCopyOnWriteTable table = (HoodieCopyOnWriteTable) HoodieTable.create(metaClient, config, hadoopConf);
+    HoodieSparkCopyOnWriteTable table = (HoodieSparkCopyOnWriteTable) HoodieSparkTable.create(config, context, metaClient);
 
     List<HoodieRecord> records = new ArrayList<>();
     // Approx 1150 records are written for block size of 64KB
@@ -383,7 +384,7 @@
     }
 
     // Insert new records
-    CommitActionExecutor actionExecutor = new UpsertCommitActionExecutor(jsc, config, table,
+    BaseSparkCommitActionExecutor actionExecutor = new SparkUpsertCommitActionExecutor(context, config, table,
         instantTime, jsc.parallelize(records));
     jsc.parallelize(Arrays.asList(1))
         .map(i -> actionExecutor.handleInsert(FSUtils.createNewFileIdPfx(), records.iterator()))
@@ -407,11 +408,11 @@
             .withStorageConfig(HoodieStorageConfig.newBuilder()
                 .parquetMaxFileSize(1000 * 1024).hfileMaxFileSize(1000 * 1024).build()).build();
     metaClient = HoodieTableMetaClient.reload(metaClient);
-    final HoodieCopyOnWriteTable table = (HoodieCopyOnWriteTable) HoodieTable.create(metaClient, config, hadoopConf);
+    final HoodieSparkCopyOnWriteTable table = (HoodieSparkCopyOnWriteTable) HoodieSparkTable.create(config, context, metaClient);
     String instantTime = "000";
     // Perform inserts of 100 records to test CreateHandle and BufferedExecutor
     final List<HoodieRecord> inserts = dataGen.generateInsertsWithHoodieAvroPayload(instantTime, 100);
-    CommitActionExecutor actionExecutor = new InsertCommitActionExecutor(jsc, config, table,
+    BaseSparkCommitActionExecutor actionExecutor = new SparkInsertCommitActionExecutor(context, config, table,
         instantTime, jsc.parallelize(inserts));
     final List<List<WriteStatus>> ws = jsc.parallelize(Arrays.asList(1)).map(x -> {
       return actionExecutor.handleInsert(UUID.randomUUID().toString(), inserts.iterator());
@@ -424,7 +425,7 @@
 
     String partitionPath = writeStatus.getPartitionPath();
     long numRecordsInPartition = updates.stream().filter(u -> u.getPartitionPath().equals(partitionPath)).count();
-    CommitActionExecutor newActionExecutor = new UpsertCommitActionExecutor(jsc, config, table,
+    BaseSparkCommitActionExecutor newActionExecutor = new SparkUpsertCommitActionExecutor(context, config, table,
         instantTime, jsc.parallelize(updates));
     final List<List<WriteStatus>> updateStatus = jsc.parallelize(Arrays.asList(1)).map(x -> {
       return newActionExecutor.handleUpdate(partitionPath, fileId, updates.iterator());
@@ -437,16 +438,16 @@
         .withPath(basePath).withSchema(TRIP_EXAMPLE_SCHEMA)
         .withBulkInsertParallelism(2).withBulkInsertSortMode(bulkInsertMode).build();
     String instantTime = makeNewCommitTime();
-    HoodieWriteClient writeClient = getHoodieWriteClient(config);
+    SparkRDDWriteClient writeClient = getHoodieWriteClient(config);
     writeClient.startCommitWithTime(instantTime);
     metaClient = HoodieTableMetaClient.reload(metaClient);
-    HoodieCopyOnWriteTable table = (HoodieCopyOnWriteTable) HoodieTable.create(metaClient, config, hadoopConf);
+    HoodieSparkCopyOnWriteTable table = (HoodieSparkCopyOnWriteTable) HoodieSparkTable.create(config, context, metaClient);
 
     // Insert new records
     final JavaRDD<HoodieRecord> inputRecords = generateTestRecordsForBulkInsert(jsc);
-    BulkInsertCommitActionExecutor bulkInsertExecutor = new BulkInsertCommitActionExecutor(
-        jsc, config, table, instantTime, inputRecords, Option.empty());
-    List<WriteStatus> returnedStatuses = bulkInsertExecutor.execute().getWriteStatuses().collect();
+    SparkBulkInsertCommitActionExecutor bulkInsertExecutor = new SparkBulkInsertCommitActionExecutor(
+        context, config, table, instantTime, inputRecords, Option.empty());
+    List<WriteStatus> returnedStatuses = ((JavaRDD<WriteStatus>)bulkInsertExecutor.execute().getWriteStatuses()).collect();
     verifyStatusResult(returnedStatuses, generateExpectedPartitionNumRecords(inputRecords));
   }
 
diff --git a/hudi-client/src/test/java/org/apache/hudi/table/action/commit/TestDeleteHelper.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestDeleteHelper.java
similarity index 85%
rename from hudi-client/src/test/java/org/apache/hudi/table/action/commit/TestDeleteHelper.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestDeleteHelper.java
index 8fda8ae..d9dc6ac 100644
--- a/hudi-client/src/test/java/org/apache/hudi/table/action/commit/TestDeleteHelper.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestDeleteHelper.java
@@ -17,23 +17,25 @@
 
 package org.apache.hudi.table.action.commit;
 
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.model.EmptyHoodieRecordPayload;
 import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.config.HoodieWriteConfig;
-import org.apache.hudi.index.bloom.HoodieBloomIndex;
+import org.apache.hudi.index.bloom.SparkHoodieBloomIndex;
 import org.apache.hudi.table.HoodieTable;
 import org.apache.hudi.table.action.HoodieWriteMetadata;
+
 import org.apache.spark.Partition;
 import org.apache.spark.api.java.JavaPairRDD;
 import org.apache.spark.api.java.JavaRDD;
 import org.apache.spark.api.java.JavaSparkContext;
-
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.extension.ExtendWith;
 import org.mockito.Mock;
 import org.mockito.junit.jupiter.MockitoExtension;
-import scala.Tuple2;
 
 import java.util.Collections;
 import java.util.List;
@@ -62,12 +64,13 @@
   private static final boolean WITHOUT_COMBINE = false;
   private static final int DELETE_PARALLELISM = 200;
 
-  @Mock private HoodieBloomIndex index;
-  @Mock private HoodieTable<EmptyHoodieRecordPayload> table;
-  @Mock private CommitActionExecutor<EmptyHoodieRecordPayload> executor;
+  @Mock private SparkHoodieBloomIndex index;
+  @Mock private HoodieTable<EmptyHoodieRecordPayload,JavaRDD<HoodieRecord>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table;
+  @Mock private BaseSparkCommitActionExecutor<EmptyHoodieRecordPayload> executor;
   @Mock private HoodieWriteMetadata metadata;
   @Mock private JavaPairRDD keyPairs;
   @Mock private JavaSparkContext jsc;
+  @Mock private HoodieSparkEngineContext context;
 
   private JavaRDD<HoodieKey> rddToDelete;
   private HoodieWriteConfig config;
@@ -75,6 +78,7 @@
   @BeforeEach
   public void setUp() {
     when(table.getIndex()).thenReturn(index);
+    when(context.getJavaSparkContext()).thenReturn(jsc);
   }
 
   @Test
@@ -82,7 +86,7 @@
     rddToDelete = mockEmptyHoodieKeyRdd();
     config = newWriteConfig(WITHOUT_COMBINE);
 
-    DeleteHelper.execute("test-time", rddToDelete, jsc, config, table, executor);
+    SparkDeleteHelper.newInstance().execute("test-time", rddToDelete, context, config, table, executor);
 
     verify(rddToDelete, never()).repartition(DELETE_PARALLELISM);
     verifyNoDeleteExecution();
@@ -93,7 +97,7 @@
     rddToDelete = newHoodieKeysRddMock(2, CombineTestMode.None);
     config = newWriteConfig(WITHOUT_COMBINE);
 
-    DeleteHelper.execute("test-time", rddToDelete, jsc, config, table, executor);
+    SparkDeleteHelper.newInstance().execute("test-time", rddToDelete, context, config, table, executor);
 
     verify(rddToDelete, times(1)).repartition(DELETE_PARALLELISM);
     verifyDeleteExecution();
@@ -104,7 +108,7 @@
     rddToDelete = newHoodieKeysRddMock(2, CombineTestMode.NoneGlobalIndex);
     config = newWriteConfig(WITH_COMBINE);
 
-    DeleteHelper.execute("test-time", rddToDelete, jsc, config, table, executor);
+    SparkDeleteHelper.newInstance().execute("test-time", rddToDelete, context, config, table, executor);
 
     verify(rddToDelete, times(1)).distinct(DELETE_PARALLELISM);
     verifyDeleteExecution();
@@ -116,7 +120,7 @@
     config = newWriteConfig(WITH_COMBINE);
     when(index.isGlobal()).thenReturn(true);
 
-    DeleteHelper.execute("test-time", rddToDelete, jsc, config, table, executor);
+    SparkDeleteHelper.newInstance().execute("test-time", rddToDelete, context, config, table, executor);
 
     verify(keyPairs, times(1)).reduceByKey(any(), eq(DELETE_PARALLELISM));
     verifyDeleteExecution();
@@ -171,10 +175,6 @@
     doReturn(Collections.emptyList()).when(emptyRdd).partitions();
     doReturn(emptyRdd).when(emptyRdd).map(any());
 
-    JavaPairRDD<Tuple2, Long> emptyPairRdd = mock(JavaPairRDD.class);
-    doReturn(Collections.emptyMap()).when(emptyPairRdd).countByKey();
-    doReturn(emptyPairRdd).when(emptyRdd).mapToPair(any());
-
     doReturn(emptyRdd).when(index).tagLocation(any(), any(), any());
     doReturn(emptyRdd).when(emptyRdd).filter(any());
 
diff --git a/hudi-client/src/test/java/org/apache/hudi/table/action/commit/TestUpsertPartitioner.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestUpsertPartitioner.java
similarity index 95%
rename from hudi-client/src/test/java/org/apache/hudi/table/action/commit/TestUpsertPartitioner.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestUpsertPartitioner.java
index 2fc9fe8..6b3426b 100644
--- a/hudi-client/src/test/java/org/apache/hudi/table/action/commit/TestUpsertPartitioner.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestUpsertPartitioner.java
@@ -31,8 +31,8 @@
 import org.apache.hudi.config.HoodieCompactionConfig;
 import org.apache.hudi.config.HoodieStorageConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
-import org.apache.hudi.table.HoodieCopyOnWriteTable;
-import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.table.HoodieSparkCopyOnWriteTable;
+import org.apache.hudi.table.HoodieSparkTable;
 import org.apache.hudi.table.WorkloadProfile;
 import org.apache.hudi.testutils.HoodieClientTestBase;
 
@@ -77,7 +77,7 @@
     FileCreateUtils.createCommit(basePath, "001");
     FileCreateUtils.createBaseFile(basePath, testPartitionPath, "001", "file1", fileSize);
     metaClient = HoodieTableMetaClient.reload(metaClient);
-    HoodieCopyOnWriteTable table = (HoodieCopyOnWriteTable) HoodieTable.create(metaClient, config, hadoopConf);
+    HoodieSparkCopyOnWriteTable table = (HoodieSparkCopyOnWriteTable) HoodieSparkTable.create(config, context, metaClient);
 
     HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(new String[] {testPartitionPath});
     List<HoodieRecord> insertRecords = dataGenerator.generateInserts("001", numInserts);
@@ -90,8 +90,8 @@
     List<HoodieRecord> records = new ArrayList<>();
     records.addAll(insertRecords);
     records.addAll(updateRecords);
-    WorkloadProfile profile = new WorkloadProfile(jsc.parallelize(records));
-    UpsertPartitioner partitioner = new UpsertPartitioner(profile, jsc, table, config);
+    WorkloadProfile profile = new WorkloadProfile(buildProfile(jsc.parallelize(records)));
+    UpsertPartitioner partitioner = new UpsertPartitioner(profile, context, table, config);
     assertEquals(0, partitioner.getPartition(
         new Tuple2<>(updateRecords.get(0).getKey(), Option.ofNullable(updateRecords.get(0).getCurrentLocation()))),
         "Update record should have gone to the 1 update partition");
@@ -196,12 +196,12 @@
 
     FileCreateUtils.createCommit(basePath, "001");
     metaClient = HoodieTableMetaClient.reload(metaClient);
-    HoodieCopyOnWriteTable table = (HoodieCopyOnWriteTable) HoodieTable.create(metaClient, config, hadoopConf);
+    HoodieSparkCopyOnWriteTable table = (HoodieSparkCopyOnWriteTable) HoodieSparkTable.create(config, context, metaClient);
     HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(new String[] {testPartitionPath});
     List<HoodieRecord> insertRecords = dataGenerator.generateInserts("001", totalInsertNum);
 
-    WorkloadProfile profile = new WorkloadProfile(jsc.parallelize(insertRecords));
-    UpsertPartitioner partitioner = new UpsertPartitioner(profile, jsc, table, config);
+    WorkloadProfile profile = new WorkloadProfile(buildProfile(jsc.parallelize(insertRecords)));
+    UpsertPartitioner partitioner = new UpsertPartitioner(profile, context, table, config);
     List<InsertBucketCumulativeWeightPair> insertBuckets = partitioner.getInsertBuckets(testPartitionPath);
 
     float bucket0Weight = 0.2f;
diff --git a/hudi-client/src/test/java/org/apache/hudi/table/action/compact/CompactionTestBase.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/CompactionTestBase.java
similarity index 96%
rename from hudi-client/src/test/java/org/apache/hudi/table/action/compact/CompactionTestBase.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/CompactionTestBase.java
index bd8c8c8..d1d31f8 100644
--- a/hudi-client/src/test/java/org/apache/hudi/table/action/compact/CompactionTestBase.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/CompactionTestBase.java
@@ -21,7 +21,7 @@
 import org.apache.hudi.avro.model.HoodieCompactionOperation;
 import org.apache.hudi.avro.model.HoodieCompactionPlan;
 import org.apache.hudi.client.HoodieReadClient;
-import org.apache.hudi.client.HoodieWriteClient;
+import org.apache.hudi.client.SparkRDDWriteClient;
 import org.apache.hudi.client.WriteStatus;
 import org.apache.hudi.common.model.FileSlice;
 import org.apache.hudi.common.model.HoodieBaseFile;
@@ -102,7 +102,7 @@
     });
   }
 
-  protected List<HoodieRecord> runNextDeltaCommits(HoodieWriteClient client, final HoodieReadClient readClient, List<String> deltaInstants,
+  protected List<HoodieRecord> runNextDeltaCommits(SparkRDDWriteClient client, final HoodieReadClient readClient, List<String> deltaInstants,
                                                    List<HoodieRecord> records, HoodieWriteConfig cfg, boolean insertFirst, List<String> expPendingCompactionInstants)
       throws Exception {
 
@@ -155,20 +155,20 @@
     assertTrue(instant.isInflight(), "Instant must be marked inflight");
   }
 
-  protected void scheduleCompaction(String compactionInstantTime, HoodieWriteClient client, HoodieWriteConfig cfg) {
+  protected void scheduleCompaction(String compactionInstantTime, SparkRDDWriteClient client, HoodieWriteConfig cfg) {
     client.scheduleCompactionAtInstant(compactionInstantTime, Option.empty());
     HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
     HoodieInstant instant = metaClient.getActiveTimeline().filterPendingCompactionTimeline().lastInstant().get();
     assertEquals(compactionInstantTime, instant.getTimestamp(), "Last compaction instant must be the one set");
   }
 
-  protected void scheduleAndExecuteCompaction(String compactionInstantTime, HoodieWriteClient client, HoodieTable table,
+  protected void scheduleAndExecuteCompaction(String compactionInstantTime, SparkRDDWriteClient client, HoodieTable table,
                                             HoodieWriteConfig cfg, int expectedNumRecs, boolean hasDeltaCommitAfterPendingCompaction) throws IOException {
     scheduleCompaction(compactionInstantTime, client, cfg);
     executeCompaction(compactionInstantTime, client, table, cfg, expectedNumRecs, hasDeltaCommitAfterPendingCompaction);
   }
 
-  protected void executeCompaction(String compactionInstantTime, HoodieWriteClient client, HoodieTable table,
+  protected void executeCompaction(String compactionInstantTime, SparkRDDWriteClient client, HoodieTable table,
                                  HoodieWriteConfig cfg, int expectedNumRecs, boolean hasDeltaCommitAfterPendingCompaction) throws IOException {
 
     client.compact(compactionInstantTime);
@@ -200,7 +200,7 @@
 
   }
 
-  protected List<WriteStatus> createNextDeltaCommit(String instantTime, List<HoodieRecord> records, HoodieWriteClient client,
+  protected List<WriteStatus> createNextDeltaCommit(String instantTime, List<HoodieRecord> records, SparkRDDWriteClient client,
                                                     HoodieTableMetaClient metaClient, HoodieWriteConfig cfg, boolean skipCommit) {
     JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
 
diff --git a/hudi-client/src/test/java/org/apache/hudi/table/action/compact/TestAsyncCompaction.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestAsyncCompaction.java
similarity index 95%
rename from hudi-client/src/test/java/org/apache/hudi/table/action/compact/TestAsyncCompaction.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestAsyncCompaction.java
index 81840b9..8da1f3d 100644
--- a/hudi-client/src/test/java/org/apache/hudi/table/action/compact/TestAsyncCompaction.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestAsyncCompaction.java
@@ -21,13 +21,14 @@
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hudi.client.HoodieReadClient;
-import org.apache.hudi.client.HoodieWriteClient;
+import org.apache.hudi.client.SparkRDDWriteClient;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
 import org.apache.hudi.common.table.timeline.HoodieInstant;
 import org.apache.hudi.common.table.timeline.HoodieInstant.State;
 import org.apache.hudi.common.table.timeline.HoodieTimeline;
 import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.table.HoodieSparkTable;
 import org.apache.hudi.table.HoodieTable;
 import org.junit.jupiter.api.Test;
 
@@ -52,7 +53,7 @@
   public void testRollbackForInflightCompaction() throws Exception {
     // Rollback inflight compaction
     HoodieWriteConfig cfg = getConfig(false);
-    try (HoodieWriteClient client = getHoodieWriteClient(cfg, true);) {
+    try (SparkRDDWriteClient client = getHoodieWriteClient(cfg, true);) {
       HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath());
       String firstInstantTime = "001";
       String secondInstantTime = "004";
@@ -79,7 +80,7 @@
 
       // Reload and rollback inflight compaction
       metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
-      HoodieTable hoodieTable = HoodieTable.create(metaClient, cfg, hadoopConf);
+      HoodieTable hoodieTable = HoodieSparkTable.create(cfg, context, metaClient);
       // hoodieTable.rollback(jsc,
       //    new HoodieInstant(true, HoodieTimeline.COMPACTION_ACTION, compactionInstantTime), false);
 
@@ -114,7 +115,7 @@
 
     int numRecs = 2000;
 
-    try (HoodieWriteClient client = getHoodieWriteClient(cfg, true);) {
+    try (SparkRDDWriteClient client = getHoodieWriteClient(cfg, true);) {
       HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath());
       List<HoodieRecord> records = dataGen.generateInserts(firstInstantTime, numRecs);
       records = runNextDeltaCommits(client, readClient, Arrays.asList(firstInstantTime, secondInstantTime), records, cfg, true,
@@ -156,7 +157,7 @@
   public void testInflightCompaction() throws Exception {
     // There is inflight compaction. Subsequent compaction run must work correctly
     HoodieWriteConfig cfg = getConfig(true);
-    try (HoodieWriteClient client = getHoodieWriteClient(cfg, true);) {
+    try (SparkRDDWriteClient client = getHoodieWriteClient(cfg, true);) {
       HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath());
       String firstInstantTime = "001";
       String secondInstantTime = "004";
@@ -189,7 +190,7 @@
   public void testScheduleIngestionBeforePendingCompaction() throws Exception {
     // Case: Failure case. Latest pending compaction instant time must be earlier than this instant time
     HoodieWriteConfig cfg = getConfig(false);
-    HoodieWriteClient client = getHoodieWriteClient(cfg, true);
+    SparkRDDWriteClient client = getHoodieWriteClient(cfg, true);
     HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath());
 
     String firstInstantTime = "001";
@@ -220,7 +221,7 @@
     // Case: Failure case. Earliest ingestion inflight instant time must be later than compaction time
 
     HoodieWriteConfig cfg = getConfig(false);
-    HoodieWriteClient client = getHoodieWriteClient(cfg, true);
+    SparkRDDWriteClient client = getHoodieWriteClient(cfg, true);
     HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath());
 
     String firstInstantTime = "001";
@@ -252,7 +253,7 @@
     // Case: Failure case. Earliest ingestion inflight instant time must be later than compaction time
 
     HoodieWriteConfig cfg = getConfig(false);
-    HoodieWriteClient client = getHoodieWriteClient(cfg, true);
+    SparkRDDWriteClient client = getHoodieWriteClient(cfg, true);
     HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath());
 
     final String firstInstantTime = "001";
@@ -287,7 +288,7 @@
   public void testCompactionAfterTwoDeltaCommits() throws Exception {
     // No Delta Commits after compaction request
     HoodieWriteConfig cfg = getConfig(true);
-    try (HoodieWriteClient client = getHoodieWriteClient(cfg, true);) {
+    try (SparkRDDWriteClient client = getHoodieWriteClient(cfg, true);) {
       HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath());
       String firstInstantTime = "001";
       String secondInstantTime = "004";
@@ -308,7 +309,7 @@
   public void testInterleavedCompaction() throws Exception {
     // Case: Two delta commits before and after compaction schedule
     HoodieWriteConfig cfg = getConfig(true);
-    try (HoodieWriteClient client = getHoodieWriteClient(cfg, true);) {
+    try (SparkRDDWriteClient client = getHoodieWriteClient(cfg, true);) {
       HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath());
       String firstInstantTime = "001";
       String secondInstantTime = "004";
diff --git a/hudi-client/src/test/java/org/apache/hudi/table/action/compact/TestHoodieCompactor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestHoodieCompactor.java
similarity index 86%
rename from hudi-client/src/test/java/org/apache/hudi/table/action/compact/TestHoodieCompactor.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestHoodieCompactor.java
index 09a9cca..7655a75 100644
--- a/hudi-client/src/test/java/org/apache/hudi/table/action/compact/TestHoodieCompactor.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestHoodieCompactor.java
@@ -19,7 +19,7 @@
 package org.apache.hudi.table.action.compact;
 
 import org.apache.hudi.avro.model.HoodieCompactionPlan;
-import org.apache.hudi.client.HoodieWriteClient;
+import org.apache.hudi.client.SparkRDDWriteClient;
 import org.apache.hudi.client.WriteStatus;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.FileSlice;
@@ -40,7 +40,8 @@
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.exception.HoodieNotSupportedException;
 import org.apache.hudi.index.HoodieIndex;
-import org.apache.hudi.index.bloom.HoodieBloomIndex;
+import org.apache.hudi.index.bloom.SparkHoodieBloomIndex;
+import org.apache.hudi.table.HoodieSparkTable;
 import org.apache.hudi.table.HoodieTable;
 import org.apache.hudi.testutils.HoodieClientTestHarness;
 
@@ -104,11 +105,11 @@
   @Test
   public void testCompactionOnCopyOnWriteFail() throws Exception {
     metaClient = HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.COPY_ON_WRITE);
-    HoodieTable<?> table = HoodieTable.create(metaClient, getConfig(), hadoopConf);
+    HoodieTable table = HoodieSparkTable.create(getConfig(), context, metaClient);
     String compactionInstantTime = HoodieActiveTimeline.createNewInstantTime();
     assertThrows(HoodieNotSupportedException.class, () -> {
-      table.scheduleCompaction(jsc, compactionInstantTime, Option.empty());
-      table.compact(jsc, compactionInstantTime);
+      table.scheduleCompaction(context, compactionInstantTime, Option.empty());
+      table.compact(context, compactionInstantTime);
     });
   }
 
@@ -116,8 +117,8 @@
   public void testCompactionEmpty() throws Exception {
     HoodieWriteConfig config = getConfig();
     metaClient = HoodieTableMetaClient.reload(metaClient);
-    HoodieTable table = HoodieTable.create(metaClient, config, hadoopConf);
-    try (HoodieWriteClient writeClient = getHoodieWriteClient(config);) {
+    HoodieTable table = HoodieSparkTable.create(getConfig(), context, metaClient);
+    try (SparkRDDWriteClient writeClient = getHoodieWriteClient(config);) {
 
       String newCommitTime = writeClient.startCommit();
       List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
@@ -125,7 +126,7 @@
       writeClient.insert(recordsRDD, newCommitTime).collect();
 
       String compactionInstantTime = HoodieActiveTimeline.createNewInstantTime();
-      Option<HoodieCompactionPlan> plan = table.scheduleCompaction(jsc, compactionInstantTime, Option.empty());
+      Option<HoodieCompactionPlan> plan = table.scheduleCompaction(context, compactionInstantTime, Option.empty());
       assertFalse(plan.isPresent(), "If there is nothing to compact, result will be empty");
     }
   }
@@ -134,7 +135,7 @@
   public void testWriteStatusContentsAfterCompaction() throws Exception {
     // insert 100 records
     HoodieWriteConfig config = getConfig();
-    try (HoodieWriteClient writeClient = getHoodieWriteClient(config)) {
+    try (SparkRDDWriteClient writeClient = getHoodieWriteClient(config)) {
       String newCommitTime = "100";
       writeClient.startCommitWithTime(newCommitTime);
 
@@ -143,14 +144,14 @@
       writeClient.insert(recordsRDD, newCommitTime).collect();
 
       // Update all the 100 records
-      HoodieTable table = HoodieTable.create(config, hadoopConf);
+      HoodieTable table = HoodieSparkTable.create(config, context);
       newCommitTime = "101";
       writeClient.startCommitWithTime(newCommitTime);
 
       List<HoodieRecord> updatedRecords = dataGen.generateUpdates(newCommitTime, records);
       JavaRDD<HoodieRecord> updatedRecordsRDD = jsc.parallelize(updatedRecords, 1);
-      HoodieIndex index = new HoodieBloomIndex<>(config);
-      updatedRecords = index.tagLocation(updatedRecordsRDD, jsc, table).collect();
+      HoodieIndex index = new SparkHoodieBloomIndex<>(config);
+      updatedRecords = ((JavaRDD<HoodieRecord>)index.tagLocation(updatedRecordsRDD, context, table)).collect();
 
       // Write them to corresponding avro logfiles. Also, set the state transition properly.
       HoodieTestUtils.writeRecordsToLogFiles(fs, metaClient.getBasePath(),
@@ -161,7 +162,7 @@
       metaClient.reloadActiveTimeline();
 
       // Verify that all data file has one log file
-      table = HoodieTable.create(config, hadoopConf);
+      table = HoodieSparkTable.create(config, context);
       for (String partitionPath : dataGen.getPartitionPaths()) {
         List<FileSlice> groupedLogFiles =
             table.getSliceView().getLatestFileSlices(partitionPath).collect(Collectors.toList());
@@ -174,11 +175,11 @@
       createInflightDeltaCommit(basePath, newCommitTime);
 
       // Do a compaction
-      table = HoodieTable.create(config, hadoopConf);
+      table = HoodieSparkTable.create(config, context);
       String compactionInstantTime = "102";
-      table.scheduleCompaction(jsc, compactionInstantTime, Option.empty());
+      table.scheduleCompaction(context, compactionInstantTime, Option.empty());
       table.getMetaClient().reloadActiveTimeline();
-      JavaRDD<WriteStatus> result = table.compact(jsc, compactionInstantTime).getWriteStatuses();
+      JavaRDD<WriteStatus> result = (JavaRDD<WriteStatus>) table.compact(context, compactionInstantTime).getWriteStatuses();
 
       // Verify that all partition paths are present in the WriteStatus result
       for (String partitionPath : dataGen.getPartitionPaths()) {
diff --git a/hudi-client/src/test/java/org/apache/hudi/table/action/compact/TestInlineCompaction.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestInlineCompaction.java
similarity index 93%
rename from hudi-client/src/test/java/org/apache/hudi/table/action/compact/TestInlineCompaction.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestInlineCompaction.java
index 4cbb461..066a965 100644
--- a/hudi-client/src/test/java/org/apache/hudi/table/action/compact/TestInlineCompaction.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestInlineCompaction.java
@@ -19,7 +19,7 @@
 package org.apache.hudi.table.action.compact;
 
 import org.apache.hudi.client.HoodieReadClient;
-import org.apache.hudi.client.HoodieWriteClient;
+import org.apache.hudi.client.SparkRDDWriteClient;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
 import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
@@ -50,7 +50,7 @@
   public void testCompactionIsNotScheduledEarly() throws Exception {
     // Given: make two commits
     HoodieWriteConfig cfg = getConfigForInlineCompaction(3);
-    try (HoodieWriteClient<?> writeClient = getHoodieWriteClient(cfg)) {
+    try (SparkRDDWriteClient<?> writeClient = getHoodieWriteClient(cfg)) {
       List<HoodieRecord> records = dataGen.generateInserts("000", 100);
       HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath());
       runNextDeltaCommits(writeClient, readClient, Arrays.asList("000", "001"), records, cfg, true, new ArrayList<>());
@@ -67,7 +67,7 @@
     HoodieWriteConfig cfg = getConfigForInlineCompaction(3);
     List<String> instants = IntStream.range(0, 2).mapToObj(i -> HoodieActiveTimeline.createNewInstantTime()).collect(Collectors.toList());
 
-    try (HoodieWriteClient<?> writeClient = getHoodieWriteClient(cfg)) {
+    try (SparkRDDWriteClient<?> writeClient = getHoodieWriteClient(cfg)) {
       List<HoodieRecord> records = dataGen.generateInserts(instants.get(0), 100);
       HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath());
       runNextDeltaCommits(writeClient, readClient, instants, records, cfg, true, new ArrayList<>());
@@ -92,7 +92,7 @@
             .withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1).build())
         .build();
     List<String> instants = CollectionUtils.createImmutableList("000", "001");
-    try (HoodieWriteClient<?> writeClient = getHoodieWriteClient(cfg)) {
+    try (SparkRDDWriteClient<?> writeClient = getHoodieWriteClient(cfg)) {
       List<HoodieRecord> records = dataGen.generateInserts(instants.get(0), 100);
       HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath());
       runNextDeltaCommits(writeClient, readClient, instants, records, cfg, true, new ArrayList<>());
@@ -103,7 +103,7 @@
 
     // When: a third commit happens
     HoodieWriteConfig inlineCfg = getConfigForInlineCompaction(2);
-    try (HoodieWriteClient<?> writeClient = getHoodieWriteClient(inlineCfg)) {
+    try (SparkRDDWriteClient<?> writeClient = getHoodieWriteClient(inlineCfg)) {
       HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
       createNextDeltaCommit("003", dataGen.generateUpdates("003", 100), writeClient, metaClient, inlineCfg, false);
     }
diff --git a/hudi-client/src/test/java/org/apache/hudi/table/action/compact/strategy/TestHoodieCompactionStrategy.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/strategy/TestHoodieCompactionStrategy.java
similarity index 100%
rename from hudi-client/src/test/java/org/apache/hudi/table/action/compact/strategy/TestHoodieCompactionStrategy.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/strategy/TestHoodieCompactionStrategy.java
diff --git a/hudi-client/src/test/java/org/apache/hudi/table/action/rollback/HoodieClientRollbackTestBase.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/HoodieClientRollbackTestBase.java
similarity index 97%
rename from hudi-client/src/test/java/org/apache/hudi/table/action/rollback/HoodieClientRollbackTestBase.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/HoodieClientRollbackTestBase.java
index ed97f43..eb0e871 100644
--- a/hudi-client/src/test/java/org/apache/hudi/table/action/rollback/HoodieClientRollbackTestBase.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/HoodieClientRollbackTestBase.java
@@ -18,7 +18,7 @@
 
 package org.apache.hudi.table.action.rollback;
 
-import org.apache.hudi.client.HoodieWriteClient;
+import org.apache.hudi.client.SparkRDDWriteClient;
 import org.apache.hudi.client.WriteStatus;
 import org.apache.hudi.common.model.FileSlice;
 import org.apache.hudi.common.model.HoodieFileGroup;
@@ -50,7 +50,7 @@
     dataGen = new HoodieTestDataGenerator(new String[]{DEFAULT_FIRST_PARTITION_PATH, DEFAULT_SECOND_PARTITION_PATH});
     //1. prepare data
     HoodieTestDataGenerator.writePartitionMetadata(fs, new String[]{DEFAULT_FIRST_PARTITION_PATH, DEFAULT_SECOND_PARTITION_PATH}, basePath);
-    HoodieWriteClient client = getHoodieWriteClient(cfg);
+    SparkRDDWriteClient client = getHoodieWriteClient(cfg);
     /**
      * Write 1 (only inserts)
      */
diff --git a/hudi-client/src/test/java/org/apache/hudi/table/action/rollback/TestCopyOnWriteRollbackActionExecutor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestCopyOnWriteRollbackActionExecutor.java
similarity index 93%
rename from hudi-client/src/test/java/org/apache/hudi/table/action/rollback/TestCopyOnWriteRollbackActionExecutor.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestCopyOnWriteRollbackActionExecutor.java
index a5bf032..e14dbf9 100644
--- a/hudi-client/src/test/java/org/apache/hudi/table/action/rollback/TestCopyOnWriteRollbackActionExecutor.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestCopyOnWriteRollbackActionExecutor.java
@@ -83,8 +83,8 @@
     HoodieInstant needRollBackInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "002");
 
     // execute CopyOnWriteRollbackActionExecutor with filelisting mode
-    CopyOnWriteRollbackActionExecutor copyOnWriteRollbackActionExecutor = new CopyOnWriteRollbackActionExecutor(jsc, table.getConfig(), table, "003", needRollBackInstant, true);
-    assertFalse(copyOnWriteRollbackActionExecutor.getRollbackStrategy() instanceof MarkerBasedRollbackStrategy);
+    SparkCopyOnWriteRollbackActionExecutor copyOnWriteRollbackActionExecutor = new SparkCopyOnWriteRollbackActionExecutor(context, table.getConfig(), table, "003", needRollBackInstant, true);
+    assertFalse(copyOnWriteRollbackActionExecutor.getRollbackStrategy() instanceof SparkMarkerBasedRollbackStrategy);
     List<HoodieRollbackStat> hoodieRollbackStats = copyOnWriteRollbackActionExecutor.executeRollback();
 
     // assert hoodieRollbackStats
@@ -133,7 +133,7 @@
     List<FileSlice> secondPartitionCommit2FileSlices = new ArrayList<>();
     HoodieWriteConfig cfg = getConfigBuilder().withRollbackUsingMarkers(isUsingMarkers).withAutoCommit(false).build();
     this.twoUpsertCommitDataWithTwoPartitions(firstPartitionCommit2FileSlices, secondPartitionCommit2FileSlices, cfg, !isUsingMarkers);
-    HoodieTable<?> table = this.getHoodieTable(metaClient, cfg);
+    HoodieTable table = this.getHoodieTable(metaClient, cfg);
 
     //2. rollback
     HoodieInstant commitInstant;
@@ -143,11 +143,11 @@
       commitInstant = table.getCompletedCommitTimeline().lastInstant().get();
     }
 
-    CopyOnWriteRollbackActionExecutor copyOnWriteRollbackActionExecutor = new CopyOnWriteRollbackActionExecutor(jsc, cfg, table, "003", commitInstant, false);
+    SparkCopyOnWriteRollbackActionExecutor copyOnWriteRollbackActionExecutor = new SparkCopyOnWriteRollbackActionExecutor(context, cfg, table, "003", commitInstant, false);
     if (!isUsingMarkers) {
-      assertFalse(copyOnWriteRollbackActionExecutor.getRollbackStrategy() instanceof MarkerBasedRollbackStrategy);
+      assertFalse(copyOnWriteRollbackActionExecutor.getRollbackStrategy() instanceof SparkMarkerBasedRollbackStrategy);
     } else {
-      assertTrue(copyOnWriteRollbackActionExecutor.getRollbackStrategy() instanceof MarkerBasedRollbackStrategy);
+      assertTrue(copyOnWriteRollbackActionExecutor.getRollbackStrategy() instanceof SparkMarkerBasedRollbackStrategy);
     }
     Map<String, HoodieRollbackPartitionMetadata> rollbackMetadata = copyOnWriteRollbackActionExecutor.execute().getPartitionMetadata();
 
diff --git a/hudi-client/src/test/java/org/apache/hudi/table/action/rollback/TestMarkerBasedRollbackStrategy.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMarkerBasedRollbackStrategy.java
similarity index 93%
rename from hudi-client/src/test/java/org/apache/hudi/table/action/rollback/TestMarkerBasedRollbackStrategy.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMarkerBasedRollbackStrategy.java
index 8c4da54..4ab189a 100644
--- a/hudi-client/src/test/java/org/apache/hudi/table/action/rollback/TestMarkerBasedRollbackStrategy.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMarkerBasedRollbackStrategy.java
@@ -24,7 +24,7 @@
 import org.apache.hudi.common.table.timeline.HoodieInstant;
 import org.apache.hudi.common.table.timeline.HoodieTimeline;
 import org.apache.hudi.common.testutils.HoodieTestTable;
-import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.table.HoodieSparkTable;
 import org.apache.hudi.testutils.HoodieClientTestBase;
 
 import org.apache.hadoop.fs.FileStatus;
@@ -69,7 +69,7 @@
         .withMarkerFile("partA", f2, IOType.CREATE);
 
     // when
-    List<HoodieRollbackStat> stats = new MarkerBasedRollbackStrategy(HoodieTable.create(metaClient, getConfig(), hadoopConf), jsc, getConfig(), "002")
+    List<HoodieRollbackStat> stats = new SparkMarkerBasedRollbackStrategy(HoodieSparkTable.create(getConfig(), context, metaClient), context, getConfig(), "002")
         .execute(new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "001"));
 
     // then: ensure files are deleted correctly, non-existent files reported as failed deletes
@@ -102,7 +102,7 @@
         .withMarkerFile("partB", f4, IOType.APPEND);
 
     // when
-    List<HoodieRollbackStat> stats = new MarkerBasedRollbackStrategy(HoodieTable.create(metaClient, getConfig(), hadoopConf), jsc, getConfig(), "002")
+    List<HoodieRollbackStat> stats = new SparkMarkerBasedRollbackStrategy(HoodieSparkTable.create(getConfig(), context, metaClient), context, getConfig(), "002")
         .execute(new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "001"));
 
     // then: ensure files are deleted, rollback block is appended (even if append does not exist)
diff --git a/hudi-client/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java
similarity index 95%
rename from hudi-client/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java
index 1edebcd..f64ce8b 100644
--- a/hudi-client/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java
@@ -88,8 +88,8 @@
 
     //2. rollback
     HoodieInstant rollBackInstant = new HoodieInstant(isUsingMarkers, HoodieTimeline.DELTA_COMMIT_ACTION, "002");
-    MergeOnReadRollbackActionExecutor mergeOnReadRollbackActionExecutor = new MergeOnReadRollbackActionExecutor(
-        jsc,
+    SparkMergeOnReadRollbackActionExecutor mergeOnReadRollbackActionExecutor = new SparkMergeOnReadRollbackActionExecutor(
+        context,
         cfg,
         table,
         "003",
@@ -97,9 +97,9 @@
         true);
     // assert is filelist mode
     if (!isUsingMarkers) {
-      assertFalse(mergeOnReadRollbackActionExecutor.getRollbackStrategy() instanceof MarkerBasedRollbackStrategy);
+      assertFalse(mergeOnReadRollbackActionExecutor.getRollbackStrategy() instanceof SparkMarkerBasedRollbackStrategy);
     } else {
-      assertTrue(mergeOnReadRollbackActionExecutor.getRollbackStrategy() instanceof MarkerBasedRollbackStrategy);
+      assertTrue(mergeOnReadRollbackActionExecutor.getRollbackStrategy() instanceof SparkMarkerBasedRollbackStrategy);
     }
 
     //3. assert the rollback stat
@@ -144,8 +144,8 @@
   public void testFailForCompletedInstants() {
     Assertions.assertThrows(IllegalArgumentException.class, () -> {
       HoodieInstant rollBackInstant = new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "002");
-      new MergeOnReadRollbackActionExecutor(
-              jsc,
+      new SparkMergeOnReadRollbackActionExecutor(
+              context,
               getConfigBuilder().build(),
               getHoodieTable(metaClient, getConfigBuilder().build()),
               "003",
diff --git a/hudi-client/src/test/java/org/apache/hudi/table/action/rollback/TestRollbackUtils.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestRollbackUtils.java
similarity index 100%
rename from hudi-client/src/test/java/org/apache/hudi/table/action/rollback/TestRollbackUtils.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestRollbackUtils.java
diff --git a/hudi-client/src/test/java/org/apache/hudi/table/upgrade/TestUpgradeDowngrade.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/upgrade/TestUpgradeDowngrade.java
similarity index 95%
rename from hudi-client/src/test/java/org/apache/hudi/table/upgrade/TestUpgradeDowngrade.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/upgrade/TestUpgradeDowngrade.java
index e1dc4ce..b8e02b9 100644
--- a/hudi-client/src/test/java/org/apache/hudi/table/upgrade/TestUpgradeDowngrade.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/upgrade/TestUpgradeDowngrade.java
@@ -18,7 +18,7 @@
 
 package org.apache.hudi.table.upgrade;
 
-import org.apache.hudi.client.HoodieWriteClient;
+import org.apache.hudi.client.SparkRDDWriteClient;
 import org.apache.hudi.client.WriteStatus;
 import org.apache.hudi.common.model.FileSlice;
 import org.apache.hudi.common.model.HoodieFileGroup;
@@ -69,7 +69,7 @@
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
 /**
- * Unit tests {@link UpgradeDowngrade}.
+ * Unit tests {@link SparkUpgradeDowngrade}.
  */
 public class TestUpgradeDowngrade extends HoodieClientTestBase {
 
@@ -110,14 +110,14 @@
       metaClient = HoodieTestUtils.init(dfs.getConf(), dfsBasePath, HoodieTableType.MERGE_ON_READ);
     }
     HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).withRollbackUsingMarkers(false).withProps(params).build();
-    HoodieWriteClient client = getHoodieWriteClient(cfg);
+    SparkRDDWriteClient client = getHoodieWriteClient(cfg);
 
     // prepare data. Make 2 commits, in which 2nd is not committed.
     List<FileSlice> firstPartitionCommit2FileSlices = new ArrayList<>();
     List<FileSlice> secondPartitionCommit2FileSlices = new ArrayList<>();
     Pair<List<HoodieRecord>, List<HoodieRecord>> inputRecords = twoUpsertCommitDataWithTwoPartitions(firstPartitionCommit2FileSlices, secondPartitionCommit2FileSlices, cfg, client, false);
 
-    HoodieTable<?> table = this.getHoodieTable(metaClient, cfg);
+    HoodieTable table = this.getHoodieTable(metaClient, cfg);
     HoodieInstant commitInstant = table.getPendingCommitTimeline().lastInstant().get();
 
     // delete one of the marker files in 2nd commit if need be.
@@ -137,7 +137,7 @@
     }
 
     // should re-create marker files for 2nd commit since its pending.
-    UpgradeDowngrade.run(metaClient, HoodieTableVersion.ONE, cfg, jsc, null);
+    new SparkUpgradeDowngrade(metaClient, cfg, context).run(metaClient, HoodieTableVersion.ONE, cfg, context, null);
 
     // assert marker files
     assertMarkerFilesForUpgrade(table, commitInstant, firstPartitionCommit2FileSlices, secondPartitionCommit2FileSlices);
@@ -152,7 +152,7 @@
     // Check the entire dataset has all records only from 1st commit and 3rd commit since 2nd is expected to be rolledback.
     assertRows(inputRecords.getKey(), thirdBatch);
     if (induceResiduesFromPrevUpgrade) {
-      assertFalse(dfs.exists(new Path(metaClient.getMetaPath(), UpgradeDowngrade.HOODIE_UPDATED_PROPERTY_FILE)));
+      assertFalse(dfs.exists(new Path(metaClient.getMetaPath(), SparkUpgradeDowngrade.HOODIE_UPDATED_PROPERTY_FILE)));
     }
   }
 
@@ -166,14 +166,14 @@
       metaClient = HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.MERGE_ON_READ);
     }
     HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).withRollbackUsingMarkers(false).withProps(params).build();
-    HoodieWriteClient client = getHoodieWriteClient(cfg);
+    SparkRDDWriteClient client = getHoodieWriteClient(cfg);
 
     // prepare data. Make 2 commits, in which 2nd is not committed.
     List<FileSlice> firstPartitionCommit2FileSlices = new ArrayList<>();
     List<FileSlice> secondPartitionCommit2FileSlices = new ArrayList<>();
     Pair<List<HoodieRecord>, List<HoodieRecord>> inputRecords = twoUpsertCommitDataWithTwoPartitions(firstPartitionCommit2FileSlices, secondPartitionCommit2FileSlices, cfg, client, false);
 
-    HoodieTable<?> table = this.getHoodieTable(metaClient, cfg);
+    HoodieTable table = this.getHoodieTable(metaClient, cfg);
     HoodieInstant commitInstant = table.getPendingCommitTimeline().lastInstant().get();
 
     // delete one of the marker files in 2nd commit if need be.
@@ -189,7 +189,7 @@
     prepForDowngrade();
 
     // downgrade should be performed. all marker files should be deleted
-    UpgradeDowngrade.run(metaClient, HoodieTableVersion.ZERO, cfg, jsc, null);
+    new SparkUpgradeDowngrade(metaClient, cfg, context).run(metaClient, HoodieTableVersion.ZERO, cfg, context, null);
 
     // assert marker files
     assertMarkerFilesForDowngrade(table, commitInstant);
@@ -285,7 +285,7 @@
       params.put(HOODIE_TABLE_TYPE_PROP_NAME, HoodieTableType.MERGE_ON_READ.name());
     }
     HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).withRollbackUsingMarkers(enableMarkedBasedRollback).withProps(params).build();
-    HoodieWriteClient client = getHoodieWriteClient(cfg, true);
+    SparkRDDWriteClient client = getHoodieWriteClient(cfg, true);
 
     client.startCommitWithTime(newCommitTime);
 
@@ -330,13 +330,13 @@
    * @param firstPartitionCommit2FileSlices list to hold file slices in first partition.
    * @param secondPartitionCommit2FileSlices list of hold file slices from second partition.
    * @param cfg instance of {@link HoodieWriteConfig}
-   * @param client instance of {@link HoodieWriteClient} to use.
+   * @param client instance of {@link SparkRDDWriteClient} to use.
    * @param commitSecondUpsert true if 2nd commit needs to be committed. false otherwise.
    * @return a pair of list of records from 1st and 2nd batch.
    */
   private Pair<List<HoodieRecord>, List<HoodieRecord>> twoUpsertCommitDataWithTwoPartitions(List<FileSlice> firstPartitionCommit2FileSlices,
       List<FileSlice> secondPartitionCommit2FileSlices,
-      HoodieWriteConfig cfg, HoodieWriteClient client,
+      HoodieWriteConfig cfg, SparkRDDWriteClient client,
       boolean commitSecondUpsert) throws IOException {
     //just generate two partitions
     dataGen = new HoodieTestDataGenerator(new String[] {DEFAULT_FIRST_PARTITION_PATH, DEFAULT_SECOND_PARTITION_PATH});
@@ -398,7 +398,7 @@
 
   private void createResidualFile() throws IOException {
     Path propertyFile = new Path(metaClient.getMetaPath() + "/" + HoodieTableConfig.HOODIE_PROPERTIES_FILE);
-    Path updatedPropertyFile = new Path(metaClient.getMetaPath() + "/" + UpgradeDowngrade.HOODIE_UPDATED_PROPERTY_FILE);
+    Path updatedPropertyFile = new Path(metaClient.getMetaPath() + "/" + SparkUpgradeDowngrade.HOODIE_UPDATED_PROPERTY_FILE);
 
     // Step1: Copy hoodie.properties to hoodie.properties.orig
     FileUtil.copy(metaClient.getFs(), propertyFile, metaClient.getFs(), updatedPropertyFile,
diff --git a/hudi-client/src/test/java/org/apache/hudi/testutils/Assertions.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/Assertions.java
similarity index 100%
rename from hudi-client/src/test/java/org/apache/hudi/testutils/Assertions.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/Assertions.java
diff --git a/hudi-client/src/test/java/org/apache/hudi/testutils/FunctionalTestHarness.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/FunctionalTestHarness.java
similarity index 90%
rename from hudi-client/src/test/java/org/apache/hudi/testutils/FunctionalTestHarness.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/FunctionalTestHarness.java
index 27db072..1020e93 100644
--- a/hudi-client/src/test/java/org/apache/hudi/testutils/FunctionalTestHarness.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/FunctionalTestHarness.java
@@ -20,13 +20,14 @@
 package org.apache.hudi.testutils;
 
 import org.apache.hudi.client.HoodieReadClient;
-import org.apache.hudi.client.HoodieWriteClient;
+import org.apache.hudi.client.SparkRDDWriteClient;
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.model.HoodieAvroPayload;
 import org.apache.hudi.common.table.HoodieTableConfig;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
 import org.apache.hudi.common.testutils.minicluster.HdfsTestService;
 import org.apache.hudi.config.HoodieWriteConfig;
-import org.apache.hudi.index.HoodieIndex;
 import org.apache.hudi.testutils.providers.DFSProvider;
 import org.apache.hudi.testutils.providers.HoodieMetaClientProvider;
 import org.apache.hudi.testutils.providers.HoodieWriteClientProvider;
@@ -58,6 +59,7 @@
   private static transient SparkSession spark;
   private static transient SQLContext sqlContext;
   private static transient JavaSparkContext jsc;
+  protected static transient HoodieSparkEngineContext context;
 
   private static transient HdfsTestService hdfsTestService;
   private static transient MiniDFSCluster dfsCluster;
@@ -104,6 +106,11 @@
     return dfs.getWorkingDirectory();
   }
 
+  @Override
+  public HoodieEngineContext context() {
+    return context;
+  }
+
   public HoodieTableMetaClient getHoodieMetaClient(Configuration hadoopConf, String basePath) throws IOException {
     return getHoodieMetaClient(hadoopConf, basePath, new Properties());
   }
@@ -118,8 +125,8 @@
   }
 
   @Override
-  public HoodieWriteClient getHoodieWriteClient(HoodieWriteConfig cfg) throws IOException {
-    return new HoodieWriteClient(jsc, cfg, false, HoodieIndex.createIndex(cfg));
+  public SparkRDDWriteClient getHoodieWriteClient(HoodieWriteConfig cfg) throws IOException {
+    return new SparkRDDWriteClient(context(), cfg, false);
   }
 
   @BeforeEach
@@ -127,11 +134,12 @@
     initialized = spark != null && hdfsTestService != null;
     if (!initialized) {
       SparkConf sparkConf = conf();
-      HoodieWriteClient.registerClasses(sparkConf);
+      SparkRDDWriteClient.registerClasses(sparkConf);
       HoodieReadClient.addHoodieSupport(sparkConf);
       spark = SparkSession.builder().config(sparkConf).getOrCreate();
       sqlContext = spark.sqlContext();
       jsc = new JavaSparkContext(spark.sparkContext());
+      context = new HoodieSparkEngineContext(jsc);
 
       hdfsTestService = new HdfsTestService();
       dfsCluster = hdfsTestService.start(true);
diff --git a/hudi-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java
similarity index 91%
rename from hudi-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java
index c176dcd..1caf9c0 100644
--- a/hudi-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java
@@ -18,7 +18,7 @@
 
 package org.apache.hudi.testutils;
 
-import org.apache.hudi.client.HoodieWriteClient;
+import org.apache.hudi.client.SparkRDDWriteClient;
 import org.apache.hudi.client.WriteStatus;
 import org.apache.hudi.common.HoodieCleanStat;
 import org.apache.hudi.common.fs.ConsistencyGuardConfig;
@@ -40,9 +40,9 @@
 import org.apache.hudi.config.HoodieIndexConfig;
 import org.apache.hudi.config.HoodieStorageConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
-import org.apache.hudi.index.HoodieIndex;
 import org.apache.hudi.index.HoodieIndex.IndexType;
-import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.index.SparkHoodieIndex;
+import org.apache.hudi.table.HoodieSparkTable;
 
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -138,8 +138,8 @@
             .withStorageType(FileSystemViewStorageType.EMBEDDED_KV_STORE).build());
   }
 
-  public HoodieTable getHoodieTable(HoodieTableMetaClient metaClient, HoodieWriteConfig config) {
-    HoodieTable table = HoodieTable.create(metaClient, config, hadoopConf);
+  public HoodieSparkTable getHoodieTable(HoodieTableMetaClient metaClient, HoodieWriteConfig config) {
+    HoodieSparkTable table = HoodieSparkTable.create(config, context, metaClient);
     ((SyncableFileSystemView) (table.getSliceView())).reset();
     return table;
   }
@@ -218,11 +218,11 @@
   private Function2<List<HoodieRecord>, String, Integer> wrapRecordsGenFunctionForPreppedCalls(
       final HoodieWriteConfig writeConfig, final Function2<List<HoodieRecord>, String, Integer> recordGenFunction) {
     return (commit, numRecords) -> {
-      final HoodieIndex index = HoodieIndex.createIndex(writeConfig);
+      final SparkHoodieIndex index = SparkHoodieIndex.createIndex(writeConfig);
       List<HoodieRecord> records = recordGenFunction.apply(commit, numRecords);
       final HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf, basePath, true);
-      HoodieTable table = HoodieTable.create(metaClient, writeConfig, hadoopConf);
-      JavaRDD<HoodieRecord> taggedRecords = index.tagLocation(jsc.parallelize(records, 1), jsc, table);
+      HoodieSparkTable table = HoodieSparkTable.create(writeConfig, context, metaClient);
+      JavaRDD<HoodieRecord> taggedRecords = index.tagLocation(jsc.parallelize(records, 1), context, table);
       return taggedRecords.collect();
     };
   }
@@ -239,13 +239,13 @@
   private Function<Integer, List<HoodieKey>> wrapDeleteKeysGenFunctionForPreppedCalls(
       final HoodieWriteConfig writeConfig, final Function<Integer, List<HoodieKey>> keyGenFunction) {
     return (numRecords) -> {
-      final HoodieIndex index = HoodieIndex.createIndex(writeConfig);
+      final SparkHoodieIndex index = SparkHoodieIndex.createIndex(writeConfig);
       List<HoodieKey> records = keyGenFunction.apply(numRecords);
       final HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf, basePath, true);
-      HoodieTable table = HoodieTable.create(metaClient, writeConfig, hadoopConf);
+      HoodieSparkTable table = HoodieSparkTable.create(writeConfig, context, metaClient);
       JavaRDD<HoodieRecord> recordsToDelete = jsc.parallelize(records, 1)
           .map(key -> new HoodieRecord(key, new EmptyHoodieRecordPayload()));
-      JavaRDD<HoodieRecord> taggedRecords = index.tagLocation(recordsToDelete, jsc, table);
+      JavaRDD<HoodieRecord> taggedRecords = index.tagLocation(recordsToDelete, context, table);
       return taggedRecords.map(record -> record.getKey()).collect();
     };
   }
@@ -300,9 +300,9 @@
    * @return RDD of write-status
    * @throws Exception in case of error
    */
-  public JavaRDD<WriteStatus> insertFirstBatch(HoodieWriteConfig writeConfig, HoodieWriteClient client, String newCommitTime,
+  public JavaRDD<WriteStatus> insertFirstBatch(HoodieWriteConfig writeConfig, SparkRDDWriteClient client, String newCommitTime,
       String initCommitTime, int numRecordsInThisCommit,
-      Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieRecord>, String> writeFn, boolean isPreppedAPI,
+      Function3<JavaRDD<WriteStatus>, SparkRDDWriteClient, JavaRDD<HoodieRecord>, String> writeFn, boolean isPreppedAPI,
       boolean assertForCommit, int expRecordsInThisCommit) throws Exception {
     final Function2<List<HoodieRecord>, String, Integer> recordGenFunction =
         generateWrapRecordsFn(isPreppedAPI, writeConfig, dataGen::generateInserts);
@@ -328,9 +328,9 @@
    * @return RDD of write-status
    * @throws Exception in case of error
    */
-  public JavaRDD<WriteStatus> insertBatch(HoodieWriteConfig writeConfig, HoodieWriteClient client, String newCommitTime,
+  public JavaRDD<WriteStatus> insertBatch(HoodieWriteConfig writeConfig, SparkRDDWriteClient client, String newCommitTime,
       String initCommitTime, int numRecordsInThisCommit,
-      Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieRecord>, String> writeFn, boolean isPreppedAPI,
+      Function3<JavaRDD<WriteStatus>, SparkRDDWriteClient, JavaRDD<HoodieRecord>, String> writeFn, boolean isPreppedAPI,
       boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits) throws Exception {
     final Function2<List<HoodieRecord>, String, Integer> recordGenFunction =
         generateWrapRecordsFn(isPreppedAPI, writeConfig, dataGen::generateInserts);
@@ -358,11 +358,11 @@
    * @return RDD of write-status
    * @throws Exception in case of error
    */
-  public JavaRDD<WriteStatus> updateBatch(HoodieWriteConfig writeConfig, HoodieWriteClient client, String newCommitTime,
-      String prevCommitTime, Option<List<String>> commitTimesBetweenPrevAndNew, String initCommitTime,
-      int numRecordsInThisCommit,
-      Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieRecord>, String> writeFn, boolean isPreppedAPI,
-      boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits) throws Exception {
+  public JavaRDD<WriteStatus> updateBatch(HoodieWriteConfig writeConfig, SparkRDDWriteClient client, String newCommitTime,
+                                          String prevCommitTime, Option<List<String>> commitTimesBetweenPrevAndNew, String initCommitTime,
+                                          int numRecordsInThisCommit,
+                                          Function3<JavaRDD<WriteStatus>, SparkRDDWriteClient, JavaRDD<HoodieRecord>, String> writeFn, boolean isPreppedAPI,
+                                          boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits) throws Exception {
     final Function2<List<HoodieRecord>, String, Integer> recordGenFunction =
         generateWrapRecordsFn(isPreppedAPI, writeConfig, dataGen::generateUniqueUpdates);
 
@@ -388,10 +388,10 @@
    * @return RDD of write-status
    * @throws Exception in case of error
    */
-  public JavaRDD<WriteStatus> deleteBatch(HoodieWriteConfig writeConfig, HoodieWriteClient client, String newCommitTime,
+  public JavaRDD<WriteStatus> deleteBatch(HoodieWriteConfig writeConfig, SparkRDDWriteClient client, String newCommitTime,
       String prevCommitTime, String initCommitTime,
       int numRecordsInThisCommit,
-      Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieKey>, String> deleteFn, boolean isPreppedAPI,
+      Function3<JavaRDD<WriteStatus>, SparkRDDWriteClient, JavaRDD<HoodieKey>, String> deleteFn, boolean isPreppedAPI,
       boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords) throws Exception {
     final Function<Integer, List<HoodieKey>> keyGenFunction =
         generateWrapDeleteKeysFn(isPreppedAPI, writeConfig, dataGen::generateUniqueDeletes);
@@ -418,10 +418,10 @@
    * @param expTotalCommits              Expected number of commits (including this commit)
    * @throws Exception in case of error
    */
-  public JavaRDD<WriteStatus> writeBatch(HoodieWriteClient client, String newCommitTime, String prevCommitTime,
+  public JavaRDD<WriteStatus> writeBatch(SparkRDDWriteClient client, String newCommitTime, String prevCommitTime,
       Option<List<String>> commitTimesBetweenPrevAndNew, String initCommitTime, int numRecordsInThisCommit,
       Function2<List<HoodieRecord>, String, Integer> recordGenFunction,
-      Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieRecord>, String> writeFn,
+      Function3<JavaRDD<WriteStatus>, SparkRDDWriteClient, JavaRDD<HoodieRecord>, String> writeFn,
       boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits) throws Exception {
 
     // Write 1 (only inserts)
@@ -486,10 +486,10 @@
    * @param expTotalRecords        Expected number of records when scanned
    * @throws Exception in case of error
    */
-  public JavaRDD<WriteStatus> deleteBatch(HoodieWriteClient client, String newCommitTime, String prevCommitTime,
+  public JavaRDD<WriteStatus> deleteBatch(SparkRDDWriteClient client, String newCommitTime, String prevCommitTime,
       String initCommitTime, int numRecordsInThisCommit,
       Function<Integer, List<HoodieKey>> keyGenFunction,
-      Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieKey>, String> deleteFn,
+      Function3<JavaRDD<WriteStatus>, SparkRDDWriteClient, JavaRDD<HoodieKey>, String> deleteFn,
       boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords) throws Exception {
 
     // Delete 1 (only deletes)
diff --git a/hudi-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java
similarity index 79%
rename from hudi-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java
index f1e3f17..b10781e 100644
--- a/hudi-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java
@@ -18,9 +18,12 @@
 package org.apache.hudi.testutils;
 
 import org.apache.hudi.client.HoodieReadClient;
-import org.apache.hudi.client.HoodieWriteClient;
+import org.apache.hudi.client.SparkRDDWriteClient;
 import org.apache.hudi.client.SparkTaskContextSupplier;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordLocation;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
 import org.apache.hudi.common.table.timeline.HoodieTimeline;
 import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
@@ -28,8 +31,9 @@
 import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
 import org.apache.hudi.common.testutils.HoodieTestUtils;
 import org.apache.hudi.common.testutils.minicluster.HdfsTestService;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
 import org.apache.hudi.config.HoodieWriteConfig;
-import org.apache.hudi.index.HoodieIndex;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
@@ -38,15 +42,20 @@
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hudi.table.WorkloadStat;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaRDD;
 import org.apache.spark.api.java.JavaSparkContext;
 import org.apache.spark.sql.SQLContext;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.TestInfo;
+import scala.Tuple2;
 
 import java.io.IOException;
 import java.io.Serializable;
+import java.util.HashMap;
+import java.util.Map;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 
@@ -59,13 +68,14 @@
   
   private String testMethodName;
   protected transient JavaSparkContext jsc = null;
+  protected transient HoodieSparkEngineContext context = null;
   protected transient Configuration hadoopConf = null;
   protected transient SQLContext sqlContext;
   protected transient FileSystem fs;
   protected transient HoodieTestDataGenerator dataGen = null;
   protected transient ExecutorService executorService;
   protected transient HoodieTableMetaClient metaClient;
-  protected transient HoodieWriteClient writeClient;
+  protected transient SparkRDDWriteClient writeClient;
   protected transient HoodieReadClient readClient;
   protected transient HoodieTableFileSystemView tableView;
 
@@ -123,6 +133,8 @@
 
     // SQLContext stuff
     sqlContext = new SQLContext(jsc);
+    context = new HoodieSparkEngineContext(jsc);
+    hadoopConf = context.getHadoopConf().get();
   }
 
   /**
@@ -149,6 +161,11 @@
       jsc.stop();
       jsc = null;
     }
+
+    if (context != null) {
+      LOG.info("Closing spark engine context used in previous test-case");
+      context = null;
+    }
   }
 
   /**
@@ -197,7 +214,7 @@
       throw new IllegalStateException("The Spark context has not been initialized.");
     }
 
-    metaClient = HoodieTestUtils.init(hadoopConf, basePath, getTableType());
+    metaClient = HoodieTestUtils.init(context.getHadoopConf().get(), basePath, getTableType());
   }
 
   /**
@@ -325,26 +342,21 @@
     }
   }
 
-  public HoodieWriteClient getHoodieWriteClient(HoodieWriteConfig cfg) {
+  public SparkRDDWriteClient getHoodieWriteClient(HoodieWriteConfig cfg) {
     return getHoodieWriteClient(cfg, false);
   }
 
-  public HoodieWriteClient getHoodieWriteClient(HoodieWriteConfig cfg, boolean rollbackInflightCommit) {
-    return getHoodieWriteClient(cfg, rollbackInflightCommit, HoodieIndex.createIndex(cfg));
-  }
-
   public HoodieReadClient getHoodieReadClient(String basePath) {
-    readClient = new HoodieReadClient(jsc, basePath, SQLContext.getOrCreate(jsc.sc()));
+    readClient = new HoodieReadClient(context, basePath, SQLContext.getOrCreate(jsc.sc()));
     return readClient;
   }
 
-  public HoodieWriteClient getHoodieWriteClient(HoodieWriteConfig cfg, boolean rollbackInflightCommit,
-      HoodieIndex index) {
+  public SparkRDDWriteClient getHoodieWriteClient(HoodieWriteConfig cfg, boolean rollbackInflightCommit) {
     if (null != writeClient) {
       writeClient.close();
       writeClient = null;
     }
-    writeClient = new HoodieWriteClient(jsc, cfg, rollbackInflightCommit, index);
+    writeClient = new SparkRDDWriteClient(context, cfg, rollbackInflightCommit);
     return writeClient;
   }
 
@@ -362,4 +374,38 @@
     }
     return tableView;
   }
+
+  protected Pair<HashMap<String, WorkloadStat>, WorkloadStat> buildProfile(JavaRDD<HoodieRecord> inputRecordsRDD) {
+    HashMap<String, WorkloadStat> partitionPathStatMap = new HashMap<>();
+    WorkloadStat globalStat = new WorkloadStat();
+
+    // group the records by partitionPath + currentLocation combination, count the number of
+    // records in each partition
+    Map<Tuple2<String, Option<HoodieRecordLocation>>, Long> partitionLocationCounts = inputRecordsRDD
+        .mapToPair(record -> new Tuple2<>(
+            new Tuple2<>(record.getPartitionPath(), Option.ofNullable(record.getCurrentLocation())), record))
+        .countByKey();
+
+    // count the number of both inserts and updates in each partition, update the counts to workLoadStats
+    for (Map.Entry<Tuple2<String, Option<HoodieRecordLocation>>, Long> e : partitionLocationCounts.entrySet()) {
+      String partitionPath = e.getKey()._1();
+      Long count = e.getValue();
+      Option<HoodieRecordLocation> locOption = e.getKey()._2();
+
+      if (!partitionPathStatMap.containsKey(partitionPath)) {
+        partitionPathStatMap.put(partitionPath, new WorkloadStat());
+      }
+
+      if (locOption.isPresent()) {
+        // update
+        partitionPathStatMap.get(partitionPath).addUpdates(locOption.get(), count);
+        globalStat.addUpdates(locOption.get(), count);
+      } else {
+        // insert
+        partitionPathStatMap.get(partitionPath).addInserts(count);
+        globalStat.addInserts(count);
+      }
+    }
+    return Pair.of(partitionPathStatMap, globalStat);
+  }
 }
diff --git a/hudi-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java
similarity index 100%
rename from hudi-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java
diff --git a/hudi-client/src/test/java/org/apache/hudi/testutils/HoodieMergeOnReadTestUtils.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieMergeOnReadTestUtils.java
similarity index 100%
rename from hudi-client/src/test/java/org/apache/hudi/testutils/HoodieMergeOnReadTestUtils.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieMergeOnReadTestUtils.java
diff --git a/hudi-client/src/test/java/org/apache/hudi/testutils/HoodieWriteableTestTable.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieWriteableTestTable.java
similarity index 100%
rename from hudi-client/src/test/java/org/apache/hudi/testutils/HoodieWriteableTestTable.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieWriteableTestTable.java
diff --git a/hudi-client/src/test/java/org/apache/hudi/testutils/MetadataMergeWriteStatus.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/MetadataMergeWriteStatus.java
similarity index 100%
rename from hudi-client/src/test/java/org/apache/hudi/testutils/MetadataMergeWriteStatus.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/MetadataMergeWriteStatus.java
diff --git a/hudi-client/src/test/java/org/apache/hudi/testutils/SparkDatasetTestUtils.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkDatasetTestUtils.java
similarity index 100%
rename from hudi-client/src/test/java/org/apache/hudi/testutils/SparkDatasetTestUtils.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkDatasetTestUtils.java
diff --git a/hudi-client/src/test/java/org/apache/hudi/testutils/providers/SparkProvider.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/providers/SparkProvider.java
similarity index 94%
rename from hudi-client/src/test/java/org/apache/hudi/testutils/providers/SparkProvider.java
rename to hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/providers/SparkProvider.java
index cdf5ac4..be15dc8 100644
--- a/hudi-client/src/test/java/org/apache/hudi/testutils/providers/SparkProvider.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/providers/SparkProvider.java
@@ -27,7 +27,7 @@
 import java.util.Collections;
 import java.util.Map;
 
-public interface SparkProvider {
+public interface SparkProvider extends org.apache.hudi.testutils.providers.HoodieEngineContextProvider {
 
   SparkSession spark();
 
@@ -52,4 +52,4 @@
   default SparkConf conf() {
     return conf(Collections.emptyMap());
   }
-}
+}
\ No newline at end of file
diff --git a/hudi-client/src/test/resources/log4j-surefire-quiet.properties b/hudi-client/hudi-spark-client/src/test/resources/log4j-surefire-quiet.properties
similarity index 100%
rename from hudi-client/src/test/resources/log4j-surefire-quiet.properties
rename to hudi-client/hudi-spark-client/src/test/resources/log4j-surefire-quiet.properties
diff --git a/hudi-client/src/test/resources/log4j-surefire.properties b/hudi-client/hudi-spark-client/src/test/resources/log4j-surefire.properties
similarity index 100%
rename from hudi-client/src/test/resources/log4j-surefire.properties
rename to hudi-client/hudi-spark-client/src/test/resources/log4j-surefire.properties
diff --git a/hudi-client/src/test/resources/testDataGeneratorSchema.txt b/hudi-client/hudi-spark-client/src/test/resources/testDataGeneratorSchema.txt
similarity index 100%
rename from hudi-client/src/test/resources/testDataGeneratorSchema.txt
rename to hudi-client/hudi-spark-client/src/test/resources/testDataGeneratorSchema.txt
diff --git a/hudi-client/pom.xml b/hudi-client/pom.xml
index 1a05175..1ab0479 100644
--- a/hudi-client/pom.xml
+++ b/hudi-client/pom.xml
@@ -24,294 +24,14 @@
   <modelVersion>4.0.0</modelVersion>
 
   <artifactId>hudi-client</artifactId>
-  <packaging>jar</packaging>
+  <packaging>pom</packaging>
 
   <properties>
     <main.basedir>${project.parent.basedir}</main.basedir>
   </properties>
 
-  <build>
-    <plugins>
-      <plugin>
-        <groupId>org.jacoco</groupId>
-        <artifactId>jacoco-maven-plugin</artifactId>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-jar-plugin</artifactId>
-        <executions>
-          <execution>
-            <goals>
-              <goal>test-jar</goal>
-            </goals>
-            <phase>test-compile</phase>
-          </execution>
-        </executions>
-        <configuration>
-          <skip>false</skip>
-        </configuration>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.rat</groupId>
-        <artifactId>apache-rat-plugin</artifactId>
-      </plugin>
-    </plugins>
-
-    <resources>
-      <resource>
-        <directory>src/main/resources</directory>
-      </resource>
-      <resource>
-        <directory>src/test/resources</directory>
-      </resource>
-    </resources>
-  </build>
-
-  <dependencies>
-    <!-- Hoodie -->
-    <dependency>
-      <groupId>org.apache.hudi</groupId>
-      <artifactId>hudi-common</artifactId>
-      <version>${project.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hudi</groupId>
-      <artifactId>hudi-timeline-service</artifactId>
-      <version>${project.version}</version>
-    </dependency>
-
-    <!-- Logging -->
-    <dependency>
-      <groupId>log4j</groupId>
-      <artifactId>log4j</artifactId>
-    </dependency>
-
-    <!-- Parquet -->
-    <dependency>
-      <groupId>org.apache.parquet</groupId>
-      <artifactId>parquet-avro</artifactId>
-    </dependency>
-
-    <!-- Spark -->
-    <dependency>
-      <groupId>org.apache.spark</groupId>
-      <artifactId>spark-core_${scala.binary.version}</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.spark</groupId>
-      <artifactId>spark-sql_${scala.binary.version}</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.spark</groupId>
-      <artifactId>spark-avro_${scala.binary.version}</artifactId>
-      <scope>provided</scope>
-    </dependency>
-
-    <!-- Dropwizard Metrics -->
-    <dependency>
-      <groupId>io.dropwizard.metrics</groupId>
-      <artifactId>metrics-graphite</artifactId>
-      <exclusions>
-        <exclusion>
-          <groupId>com.rabbitmq</groupId>
-          <artifactId>*</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
-      <groupId>io.dropwizard.metrics</groupId>
-      <artifactId>metrics-core</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>io.dropwizard.metrics</groupId>
-      <artifactId>metrics-jmx</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>io.prometheus</groupId>
-      <artifactId>simpleclient</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>io.prometheus</groupId>
-      <artifactId>simpleclient_httpserver</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>io.prometheus</groupId>
-      <artifactId>simpleclient_dropwizard</artifactId>
-    </dependency>
-    <dependency>
-      <groupId>io.prometheus</groupId>
-      <artifactId>simpleclient_pushgateway</artifactId>
-    </dependency>
-
-    <dependency>
-      <groupId>com.beust</groupId>
-      <artifactId>jcommander</artifactId>
-      <scope>test</scope>
-    </dependency>
-
-    <!-- Hadoop -->
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-client</artifactId>
-      <exclusions>
-        <exclusion>
-          <groupId>javax.servlet</groupId>
-          <artifactId>*</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-hdfs</artifactId>
-      <classifier>tests</classifier>
-      <!-- Need these exclusions to make sure JavaSparkContext can be setup. https://issues.apache.org/jira/browse/SPARK-1693 -->
-      <exclusions>
-        <exclusion>
-          <groupId>org.mortbay.jetty</groupId>
-          <artifactId>*</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>javax.servlet.jsp</groupId>
-          <artifactId>*</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>javax.servlet</groupId>
-          <artifactId>*</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-common</artifactId>
-      <classifier>tests</classifier>
-      <exclusions>
-        <exclusion>
-          <groupId>org.mortbay.jetty</groupId>
-          <artifactId>*</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>javax.servlet.jsp</groupId>
-          <artifactId>*</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>javax.servlet</groupId>
-          <artifactId>*</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-
-    <!-- Hbase -->
-    <dependency>
-      <groupId>org.apache.hbase</groupId>
-      <artifactId>hbase-client</artifactId>
-      <version>${hbase.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hbase</groupId>
-      <artifactId>hbase-server</artifactId>
-      <version>${hbase.version}</version>
-      <scope>provided</scope>
-    </dependency>
-
-    <!-- Hoodie - Tests -->
-    <dependency>
-      <groupId>org.apache.hudi</groupId>
-      <artifactId>hudi-common</artifactId>
-      <version>${project.version}</version>
-      <classifier>tests</classifier>
-      <type>test-jar</type>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.hudi</groupId>
-      <artifactId>hudi-hadoop-mr</artifactId>
-      <version>${project.version}</version>
-      <scope>test</scope>
-    </dependency>
-
-    <!-- HBase - Tests -->
-    <dependency>
-      <groupId>org.apache.hbase</groupId>
-      <artifactId>hbase-testing-util</artifactId>
-      <version>${hbase.version}</version>
-      <scope>test</scope>
-      <exclusions>
-        <exclusion>
-          <groupId>org.codehaus.jackson</groupId>
-          <artifactId>jackson-mapper-asl</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.codehaus.jackson</groupId>
-          <artifactId>jackson-core-asl</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>javax.xml.bind</groupId>
-          <artifactId>*</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-
-    <!-- Hive - Tests -->
-    <dependency>
-      <groupId>${hive.groupid}</groupId>
-      <artifactId>hive-exec</artifactId>
-      <version>${hive.version}</version>
-      <scope>test</scope>
-      <classifier>${hive.exec.classifier}</classifier>
-    </dependency>
-    <dependency>
-      <groupId>${hive.groupid}</groupId>
-      <artifactId>hive-metastore</artifactId>
-      <version>${hive.version}</version>
-      <scope>test</scope>
-    </dependency>
-
-    <dependency>
-      <groupId>org.junit.jupiter</groupId>
-      <artifactId>junit-jupiter-api</artifactId>
-      <scope>test</scope>
-    </dependency>
-
-    <dependency>
-      <groupId>org.junit.jupiter</groupId>
-      <artifactId>junit-jupiter-engine</artifactId>
-      <scope>test</scope>
-    </dependency>
-
-    <dependency>
-      <groupId>org.junit.vintage</groupId>
-      <artifactId>junit-vintage-engine</artifactId>
-      <scope>test</scope>
-    </dependency>
-
-    <dependency>
-      <groupId>org.junit.jupiter</groupId>
-      <artifactId>junit-jupiter-params</artifactId>
-      <scope>test</scope>
-    </dependency>
-
-    <dependency>
-      <groupId>org.mockito</groupId>
-      <artifactId>mockito-junit-jupiter</artifactId>
-      <scope>test</scope>
-    </dependency>
-
-    <dependency>
-      <groupId>org.junit.platform</groupId>
-      <artifactId>junit-platform-runner</artifactId>
-      <scope>test</scope>
-    </dependency>
-
-    <dependency>
-      <groupId>org.junit.platform</groupId>
-      <artifactId>junit-platform-suite-api</artifactId>
-      <scope>test</scope>
-    </dependency>
-
-    <dependency>
-      <groupId>org.junit.platform</groupId>
-      <artifactId>junit-platform-commons</artifactId>
-      <scope>test</scope>
-    </dependency>
-  </dependencies>
+  <modules>
+    <module>hudi-client-common</module>
+    <module>hudi-spark-client</module>
+  </modules>
 </project>
diff --git a/hudi-client/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java b/hudi-client/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
deleted file mode 100644
index 06e980d..0000000
--- a/hudi-client/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
+++ /dev/null
@@ -1,266 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hudi.client;
-
-import com.codahale.metrics.Timer;
-import org.apache.hudi.callback.HoodieWriteCommitCallback;
-import org.apache.hudi.callback.common.HoodieWriteCommitCallbackMessage;
-import org.apache.hudi.callback.util.HoodieCommitCallbackFactory;
-import org.apache.hudi.client.embedded.EmbeddedTimelineService;
-import org.apache.hudi.common.model.HoodieCommitMetadata;
-import org.apache.hudi.common.model.HoodieRecordPayload;
-import org.apache.hudi.common.model.HoodieWriteStat;
-import org.apache.hudi.common.model.WriteOperationType;
-import org.apache.hudi.common.table.HoodieTableMetaClient;
-import org.apache.hudi.common.table.HoodieTableVersion;
-import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
-import org.apache.hudi.common.table.timeline.HoodieInstant;
-import org.apache.hudi.common.table.timeline.HoodieTimeline;
-import org.apache.hudi.common.util.CommitUtils;
-import org.apache.hudi.common.util.Option;
-import org.apache.hudi.config.HoodieWriteConfig;
-import org.apache.hudi.exception.HoodieCommitException;
-import org.apache.hudi.exception.HoodieIOException;
-import org.apache.hudi.index.HoodieIndex;
-import org.apache.hudi.metrics.HoodieMetrics;
-import org.apache.hudi.table.HoodieTable;
-import org.apache.hudi.table.upgrade.UpgradeDowngrade;
-import org.apache.log4j.LogManager;
-import org.apache.log4j.Logger;
-import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
-
-import java.io.IOException;
-import java.nio.charset.StandardCharsets;
-import java.text.ParseException;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-
-/**
- * Abstract Write Client providing functionality for performing commit, index updates and rollback
- *  Reused for regular write operations like upsert/insert/bulk-insert.. as well as bootstrap
- * @param <T> Sub type of HoodieRecordPayload
- */
-public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHoodieClient {
-
-  private static final Logger LOG = LogManager.getLogger(AbstractHoodieWriteClient.class);
-
-  private final transient HoodieMetrics metrics;
-  private final transient HoodieIndex<T> index;
-
-  private transient Timer.Context writeContext = null;
-  private transient WriteOperationType operationType;
-  private transient HoodieWriteCommitCallback commitCallback;
-
-  public void setOperationType(WriteOperationType operationType) {
-    this.operationType = operationType;
-  }
-
-  public WriteOperationType getOperationType() {
-    return this.operationType;
-  }
-
-  protected AbstractHoodieWriteClient(JavaSparkContext jsc, HoodieIndex index, HoodieWriteConfig clientConfig,
-      Option<EmbeddedTimelineService> timelineServer) {
-    super(jsc, clientConfig, timelineServer);
-    this.metrics = new HoodieMetrics(config, config.getTableName());
-    this.index = index;
-  }
-
-  /**
-   * Commit changes performed at the given instantTime marker.
-   */
-  public boolean commit(String instantTime, JavaRDD<WriteStatus> writeStatuses) {
-    return commit(instantTime, writeStatuses, Option.empty());
-  }
-
-  /**
-   *
-   * Commit changes performed at the given instantTime marker.
-   */
-  public boolean commit(String instantTime, JavaRDD<WriteStatus> writeStatuses,
-                        Option<Map<String, String>> extraMetadata) {
-    HoodieTableMetaClient metaClient = createMetaClient(false);
-    String actionType = metaClient.getCommitActionType();
-    return commit(instantTime, writeStatuses, extraMetadata, actionType, Collections.emptyMap());
-  }
-
-  /**
-   * Complete changes performed at the given instantTime marker with specified action.
-   */
-  public boolean commit(String instantTime, JavaRDD<WriteStatus> writeStatuses,
-      Option<Map<String, String>> extraMetadata, String commitActionType, Map<String, List<String>> partitionToReplacedFileIds) {
-    List<HoodieWriteStat> writeStats = writeStatuses.map(WriteStatus::getStat).collect();
-    return commitStats(instantTime, writeStats, extraMetadata, commitActionType, partitionToReplacedFileIds);
-  }
-
-  public boolean commitStats(String instantTime, List<HoodieWriteStat> stats, Option<Map<String, String>> extraMetadata,
-                             String commitActionType) {
-    return commitStats(instantTime, stats, extraMetadata, commitActionType, Collections.emptyMap());
-  }
-
-  public boolean commitStats(String instantTime, List<HoodieWriteStat> stats, Option<Map<String, String>> extraMetadata,
-                             String commitActionType, Map<String, List<String>> partitionToReplaceFileIds) {
-    LOG.info("Committing " + instantTime + " action " + commitActionType);
-    // Create a Hoodie table which encapsulated the commits and files visible
-    HoodieTable<T> table = HoodieTable.create(config, hadoopConf);
-
-    HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
-    HoodieCommitMetadata metadata = CommitUtils.buildMetadata(stats, partitionToReplaceFileIds, extraMetadata, operationType, config.getSchema(), commitActionType);
-    // Finalize write
-    finalizeWrite(table, instantTime, stats);
-
-    try {
-      activeTimeline.saveAsComplete(new HoodieInstant(true, commitActionType, instantTime),
-          Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
-      postCommit(table, metadata, instantTime, extraMetadata);
-      emitCommitMetrics(instantTime, metadata, commitActionType);
-      LOG.info("Committed " + instantTime);
-    } catch (IOException e) {
-      throw new HoodieCommitException("Failed to complete commit " + config.getBasePath() + " at time " + instantTime,
-          e);
-    }
-
-    // callback if needed.
-    if (config.writeCommitCallbackOn()) {
-      if (null == commitCallback) {
-        commitCallback = HoodieCommitCallbackFactory.create(config);
-      }
-      commitCallback.call(new HoodieWriteCommitCallbackMessage(instantTime, config.getTableName(), config.getBasePath()));
-    }
-    return true;
-  }
-
-  void emitCommitMetrics(String instantTime, HoodieCommitMetadata metadata, String actionType) {
-    try {
-
-      if (writeContext != null) {
-        long durationInMs = metrics.getDurationInMs(writeContext.stop());
-        metrics.updateCommitMetrics(HoodieActiveTimeline.COMMIT_FORMATTER.parse(instantTime).getTime(), durationInMs,
-            metadata, actionType);
-        writeContext = null;
-      }
-    } catch (ParseException e) {
-      throw new HoodieCommitException("Failed to complete commit " + config.getBasePath() + " at time " + instantTime
-          + "Instant time is not of valid format", e);
-    }
-  }
-
-  /**
-   * Post Commit Hook. Derived classes use this method to perform post-commit processing
-   *
-   * @param table         table to commit on
-   * @param metadata      Commit Metadata corresponding to committed instant
-   * @param instantTime   Instant Time
-   * @param extraMetadata Additional Metadata passed by user
-   */
-  protected abstract void postCommit(HoodieTable<?> table, HoodieCommitMetadata metadata, String instantTime, Option<Map<String, String>> extraMetadata);
-
-  /**
-   * Finalize Write operation.
-   * @param table  HoodieTable
-   * @param instantTime Instant Time
-   * @param stats Hoodie Write Stat
-   */
-  protected void finalizeWrite(HoodieTable<T> table, String instantTime, List<HoodieWriteStat> stats) {
-    try {
-      final Timer.Context finalizeCtx = metrics.getFinalizeCtx();
-      table.finalizeWrite(jsc, instantTime, stats);
-      if (finalizeCtx != null) {
-        Option<Long> durationInMs = Option.of(metrics.getDurationInMs(finalizeCtx.stop()));
-        durationInMs.ifPresent(duration -> {
-          LOG.info("Finalize write elapsed time (milliseconds): " + duration);
-          metrics.updateFinalizeWriteMetrics(duration, stats.size());
-        });
-      }
-    } catch (HoodieIOException ioe) {
-      throw new HoodieCommitException("Failed to complete commit " + instantTime + " due to finalize errors.", ioe);
-    }
-  }
-
-  public HoodieMetrics getMetrics() {
-    return metrics;
-  }
-
-  public HoodieIndex<T> getIndex() {
-    return index;
-  }
-
-  /**
-   * Get HoodieTable and init {@link Timer.Context}.
-   *
-   * @param operationType write operation type
-   * @param instantTime current inflight instant time
-   * @return HoodieTable
-   */
-  protected HoodieTable getTableAndInitCtx(WriteOperationType operationType, String instantTime) {
-    HoodieTableMetaClient metaClient = createMetaClient(true);
-    UpgradeDowngrade.run(metaClient, HoodieTableVersion.current(), config, jsc, instantTime);
-    return getTableAndInitCtx(metaClient, operationType);
-  }
-
-  private HoodieTable getTableAndInitCtx(HoodieTableMetaClient metaClient, WriteOperationType operationType) {
-    if (operationType == WriteOperationType.DELETE) {
-      setWriteSchemaForDeletes(metaClient);
-    }
-    // Create a Hoodie table which encapsulated the commits and files visible
-    HoodieTable table = HoodieTable.create(metaClient, config, hadoopConf);
-    if (table.getMetaClient().getCommitActionType().equals(HoodieTimeline.COMMIT_ACTION)) {
-      writeContext = metrics.getCommitCtx();
-    } else {
-      writeContext = metrics.getDeltaCommitCtx();
-    }
-    return table;
-  }
-
-  /**
-   * Sets write schema from last instant since deletes may not have schema set in the config.
-   */
-  private void setWriteSchemaForDeletes(HoodieTableMetaClient metaClient) {
-    try {
-      HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline();
-      Option<HoodieInstant> lastInstant =
-          activeTimeline.filterCompletedInstants().filter(s -> s.getAction().equals(metaClient.getCommitActionType()))
-              .lastInstant();
-      if (lastInstant.isPresent()) {
-        HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(
-            activeTimeline.getInstantDetails(lastInstant.get()).get(), HoodieCommitMetadata.class);
-        if (commitMetadata.getExtraMetadata().containsKey(HoodieCommitMetadata.SCHEMA_KEY)) {
-          config.setSchema(commitMetadata.getExtraMetadata().get(HoodieCommitMetadata.SCHEMA_KEY));
-        } else {
-          throw new HoodieIOException("Latest commit does not have any schema in commit metadata");
-        }
-      } else {
-        throw new HoodieIOException("Deletes issued without any prior commits");
-      }
-    } catch (IOException e) {
-      throw new HoodieIOException("IOException thrown while reading last commit metadata", e);
-    }
-  }
-
-  @Override
-  public void close() {
-    // Stop timeline-server if running
-    super.close();
-    // Calling this here releases any resources used by your index, so make sure to finish any related operations
-    // before this point
-    this.index.close();
-  }
-}
diff --git a/hudi-client/src/main/java/org/apache/hudi/client/SparkTaskContextSupplier.java b/hudi-client/src/main/java/org/apache/hudi/client/SparkTaskContextSupplier.java
deleted file mode 100644
index 601dd98..0000000
--- a/hudi-client/src/main/java/org/apache/hudi/client/SparkTaskContextSupplier.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hudi.client;
-
-import org.apache.spark.TaskContext;
-
-import java.io.Serializable;
-import java.util.function.Supplier;
-
-/**
- * Spark task context supplier.
- */
-public class SparkTaskContextSupplier implements Serializable {
-
-  public Supplier<Integer> getPartitionIdSupplier() {
-    return () -> TaskContext.getPartitionId();
-  }
-
-  public Supplier<Integer> getStageIdSupplier() {
-    return () -> TaskContext.get().stageId();
-  }
-
-  public Supplier<Long> getAttemptIdSupplier() {
-    return () -> TaskContext.get().taskAttemptId();
-  }
-}
diff --git a/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/BootstrapSchemaProvider.java b/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/BootstrapSchemaProvider.java
deleted file mode 100644
index 2018ca9..0000000
--- a/hudi-client/src/main/java/org/apache/hudi/client/bootstrap/BootstrapSchemaProvider.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hudi.client.bootstrap;
-
-import org.apache.hudi.avro.HoodieAvroUtils;
-import org.apache.hudi.avro.model.HoodieFileStatus;
-import org.apache.hudi.common.bootstrap.FileStatusUtils;
-import org.apache.hudi.common.util.ParquetUtils;
-import org.apache.hudi.common.util.collection.Pair;
-import org.apache.hudi.config.HoodieWriteConfig;
-import org.apache.hudi.exception.HoodieException;
-import org.apache.parquet.schema.MessageType;
-import org.apache.spark.api.java.JavaSparkContext;
-
-import org.apache.avro.Schema;
-import org.apache.hadoop.fs.Path;
-import org.apache.spark.sql.avro.SchemaConverters;
-import org.apache.spark.sql.execution.datasources.parquet.ParquetToSparkSchemaConverter;
-import org.apache.spark.sql.internal.SQLConf;
-import org.apache.spark.sql.types.StructType;
-
-import java.util.List;
-import java.util.Objects;
-
-/**
- * Bootstrap Schema Provider. Schema provided in config is used. If not available, use schema from Parquet
- */
-public class BootstrapSchemaProvider {
-
-  protected final HoodieWriteConfig writeConfig;
-
-  public BootstrapSchemaProvider(HoodieWriteConfig writeConfig) {
-    this.writeConfig = writeConfig;
-  }
-
-  /**
-   * Main API to select avro schema for bootstrapping.
-   * @param jsc Java Spark Context
-   * @param partitions  List of partitions with files within them
-   * @return Avro Schema
-   */
-  public final Schema getBootstrapSchema(JavaSparkContext jsc, List<Pair<String, List<HoodieFileStatus>>> partitions) {
-    if (writeConfig.getSchema() != null) {
-      // Use schema specified by user if set
-      Schema userSchema = Schema.parse(writeConfig.getSchema());
-      if (!HoodieAvroUtils.getNullSchema().equals(userSchema)) {
-        return userSchema;
-      }
-    }
-    return getBootstrapSourceSchema(jsc, partitions);
-  }
-
-  /**
-   * Select a random file to be used to generate avro schema.
-   * Override this method to get custom schema selection.
-   * @param jsc Java Spark Context
-   * @param partitions  List of partitions with files within them
-   * @return Avro Schema
-   */
-  protected Schema getBootstrapSourceSchema(JavaSparkContext jsc,
-      List<Pair<String, List<HoodieFileStatus>>> partitions) {
-    MessageType parquetSchema = partitions.stream().flatMap(p -> p.getValue().stream()).map(fs -> {
-      try {
-        Path filePath = FileStatusUtils.toPath(fs.getPath());
-        return ParquetUtils.readSchema(jsc.hadoopConfiguration(), filePath);
-      } catch (Exception ex) {
-        return null;
-      }
-    }).filter(Objects::nonNull).findAny()
-        .orElseThrow(() -> new HoodieException("Could not determine schema from the data files."));
-
-
-    ParquetToSparkSchemaConverter converter = new ParquetToSparkSchemaConverter(
-            Boolean.parseBoolean(SQLConf.PARQUET_BINARY_AS_STRING().defaultValueString()),
-            Boolean.parseBoolean(SQLConf.PARQUET_INT96_AS_TIMESTAMP().defaultValueString()));
-    StructType sparkSchema = converter.convert(parquetSchema);
-    String tableName = HoodieAvroUtils.sanitizeName(writeConfig.getTableName());
-    String structName = tableName + "_record";
-    String recordNamespace = "hoodie." + tableName;
-
-    return SchemaConverters.toAvroType(sparkSchema, false, structName, recordNamespace);
-  }
-
-}
diff --git a/hudi-client/src/main/java/org/apache/hudi/client/utils/ClientUtils.java b/hudi-client/src/main/java/org/apache/hudi/client/utils/ClientUtils.java
deleted file mode 100644
index 0cfe0c1..0000000
--- a/hudi-client/src/main/java/org/apache/hudi/client/utils/ClientUtils.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hudi.client.utils;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hudi.common.table.HoodieTableMetaClient;
-import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
-import org.apache.hudi.common.util.Option;
-import org.apache.hudi.config.HoodieWriteConfig;
-
-public class ClientUtils {
-
-  /**
-   * Create Consistency Aware MetaClient.
-   *
-   * @param hadoopConf Configuration
-   * @param config HoodieWriteConfig
-   * @param loadActiveTimelineOnLoad early loading of timeline
-   */
-  public static HoodieTableMetaClient createMetaClient(Configuration hadoopConf, HoodieWriteConfig config,
-      boolean loadActiveTimelineOnLoad) {
-    return new HoodieTableMetaClient(hadoopConf, config.getBasePath(), loadActiveTimelineOnLoad,
-        config.getConsistencyGuardConfig(),
-        Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion())));
-  }
-}
diff --git a/hudi-client/src/main/java/org/apache/hudi/client/utils/SparkConfigUtils.java b/hudi-client/src/main/java/org/apache/hudi/client/utils/SparkConfigUtils.java
deleted file mode 100644
index 0a6b608..0000000
--- a/hudi-client/src/main/java/org/apache/hudi/client/utils/SparkConfigUtils.java
+++ /dev/null
@@ -1,106 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hudi.client.utils;
-
-import org.apache.hudi.config.HoodieIndexConfig;
-
-import org.apache.spark.SparkEnv;
-import org.apache.spark.storage.StorageLevel;
-import org.apache.spark.util.Utils;
-
-import java.util.Properties;
-
-import static org.apache.hudi.config.HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES;
-import static org.apache.hudi.config.HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FRACTION_FOR_COMPACTION;
-import static org.apache.hudi.config.HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FRACTION_FOR_MERGE;
-import static org.apache.hudi.config.HoodieMemoryConfig.DEFAULT_MIN_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES;
-import static org.apache.hudi.config.HoodieMemoryConfig.MAX_MEMORY_FOR_COMPACTION_PROP;
-import static org.apache.hudi.config.HoodieMemoryConfig.MAX_MEMORY_FOR_MERGE_PROP;
-import static org.apache.hudi.config.HoodieMemoryConfig.MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP;
-import static org.apache.hudi.config.HoodieMemoryConfig.MAX_MEMORY_FRACTION_FOR_MERGE_PROP;
-import static org.apache.hudi.config.HoodieWriteConfig.WRITE_STATUS_STORAGE_LEVEL;
-
-/**
- * Spark config utils.
- */
-public class SparkConfigUtils {
-
-  /**
-   * Dynamic calculation of max memory to use for for spillable map. user.available.memory = spark.executor.memory *
-   * (1 - spark.memory.fraction) spillable.available.memory = user.available.memory * hoodie.memory.fraction. Anytime
-   * the spark.executor.memory or the spark.memory.fraction is changed, the memory used for spillable map changes
-   * accordingly
-   */
-  public static long getMaxMemoryAllowedForMerge(String maxMemoryFraction) {
-    final String SPARK_EXECUTOR_MEMORY_PROP = "spark.executor.memory";
-    final String SPARK_EXECUTOR_MEMORY_FRACTION_PROP = "spark.memory.fraction";
-    // This is hard-coded in spark code {@link
-    // https://github.com/apache/spark/blob/576c43fb4226e4efa12189b41c3bc862019862c6/core/src/main/scala/org/apache/
-    // spark/memory/UnifiedMemoryManager.scala#L231} so have to re-define this here
-    final String DEFAULT_SPARK_EXECUTOR_MEMORY_FRACTION = "0.6";
-    // This is hard-coded in spark code {@link
-    // https://github.com/apache/spark/blob/576c43fb4226e4efa12189b41c3bc862019862c6/core/src/main/scala/org/apache/
-    // spark/SparkContext.scala#L471} so have to re-define this here
-    final String DEFAULT_SPARK_EXECUTOR_MEMORY_MB = "1024"; // in MB
-    if (SparkEnv.get() != null) {
-      // 1 GB is the default conf used by Spark, look at SparkContext.scala
-      long executorMemoryInBytes = Utils.memoryStringToMb(
-          SparkEnv.get().conf().get(SPARK_EXECUTOR_MEMORY_PROP, DEFAULT_SPARK_EXECUTOR_MEMORY_MB)) * 1024 * 1024L;
-      // 0.6 is the default value used by Spark,
-      // look at {@link
-      // https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/SparkConf.scala#L507}
-      double memoryFraction = Double.parseDouble(
-          SparkEnv.get().conf().get(SPARK_EXECUTOR_MEMORY_FRACTION_PROP, DEFAULT_SPARK_EXECUTOR_MEMORY_FRACTION));
-      double maxMemoryFractionForMerge = Double.parseDouble(maxMemoryFraction);
-      double userAvailableMemory = executorMemoryInBytes * (1 - memoryFraction);
-      long maxMemoryForMerge = (long) Math.floor(userAvailableMemory * maxMemoryFractionForMerge);
-      return Math.max(DEFAULT_MIN_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES, maxMemoryForMerge);
-    } else {
-      return DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES;
-    }
-  }
-
-  public static StorageLevel getWriteStatusStorageLevel(Properties properties) {
-    return StorageLevel.fromString(properties.getProperty(WRITE_STATUS_STORAGE_LEVEL));
-  }
-
-  public static StorageLevel getBloomIndexInputStorageLevel(Properties properties) {
-    return StorageLevel.fromString(properties.getProperty(HoodieIndexConfig.BLOOM_INDEX_INPUT_STORAGE_LEVEL));
-  }
-
-  public static long getMaxMemoryPerPartitionMerge(Properties properties) {
-    if (properties.containsKey(MAX_MEMORY_FOR_MERGE_PROP)) {
-      return Long.parseLong(properties.getProperty(MAX_MEMORY_FOR_MERGE_PROP));
-    }
-    String fraction = properties.getProperty(MAX_MEMORY_FRACTION_FOR_MERGE_PROP, DEFAULT_MAX_MEMORY_FRACTION_FOR_MERGE);
-    return getMaxMemoryAllowedForMerge(fraction);
-  }
-
-  public static long getMaxMemoryPerCompaction(Properties properties) {
-    if (properties.containsKey(MAX_MEMORY_FOR_COMPACTION_PROP)) {
-      return Long.parseLong(properties.getProperty(MAX_MEMORY_FOR_COMPACTION_PROP));
-    }
-    String fraction = properties.getProperty(MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP, DEFAULT_MAX_MEMORY_FRACTION_FOR_COMPACTION);
-    return getMaxMemoryAllowedForMerge(fraction);
-  }
-
-  public static StorageLevel getSimpleIndexInputStorageLevel(Properties properties) {
-    return StorageLevel.fromString(properties.getProperty(HoodieIndexConfig.SIMPLE_INDEX_INPUT_STORAGE_LEVEL));
-  }
-}
diff --git a/hudi-client/src/main/java/org/apache/hudi/execution/LazyInsertIterable.java b/hudi-client/src/main/java/org/apache/hudi/execution/LazyInsertIterable.java
deleted file mode 100644
index 572956d..0000000
--- a/hudi-client/src/main/java/org/apache/hudi/execution/LazyInsertIterable.java
+++ /dev/null
@@ -1,139 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hudi.execution;
-
-import org.apache.hudi.client.SparkTaskContextSupplier;
-import org.apache.hudi.client.WriteStatus;
-import org.apache.hudi.client.utils.LazyIterableIterator;
-import org.apache.hudi.common.model.HoodieRecord;
-import org.apache.hudi.common.model.HoodieRecordPayload;
-import org.apache.hudi.common.util.Option;
-import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor;
-import org.apache.hudi.config.HoodieWriteConfig;
-import org.apache.hudi.exception.HoodieException;
-import org.apache.hudi.io.CreateHandleFactory;
-import org.apache.hudi.io.WriteHandleFactory;
-import org.apache.hudi.table.HoodieTable;
-
-import org.apache.avro.Schema;
-import org.apache.avro.generic.IndexedRecord;
-
-import java.util.Iterator;
-import java.util.List;
-import java.util.function.Function;
-
-/**
- * Lazy Iterable, that writes a stream of HoodieRecords sorted by the partitionPath, into new files.
- */
-public class LazyInsertIterable<T extends HoodieRecordPayload>
-    extends LazyIterableIterator<HoodieRecord<T>, List<WriteStatus>> {
-
-  protected final HoodieWriteConfig hoodieConfig;
-  protected final String instantTime;
-  protected boolean areRecordsSorted;
-  protected final HoodieTable<T> hoodieTable;
-  protected final String idPrefix;
-  protected SparkTaskContextSupplier sparkTaskContextSupplier;
-  protected WriteHandleFactory<T> writeHandleFactory;
-
-  public LazyInsertIterable(Iterator<HoodieRecord<T>> sortedRecordItr, HoodieWriteConfig config,
-                            String instantTime, HoodieTable<T> hoodieTable, String idPrefix,
-                            SparkTaskContextSupplier sparkTaskContextSupplier) {
-    this(sortedRecordItr, true, config, instantTime, hoodieTable, idPrefix, sparkTaskContextSupplier);
-  }
-
-  public LazyInsertIterable(Iterator<HoodieRecord<T>> recordItr, boolean areRecordsSorted,
-                            HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
-                            String idPrefix, SparkTaskContextSupplier sparkTaskContextSupplier) {
-    this(recordItr, areRecordsSorted, config, instantTime, hoodieTable, idPrefix, sparkTaskContextSupplier,
-        new CreateHandleFactory<>());
-  }
-
-  public LazyInsertIterable(Iterator<HoodieRecord<T>> recordItr, boolean areRecordsSorted,
-                            HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
-                            String idPrefix, SparkTaskContextSupplier sparkTaskContextSupplier,
-                            WriteHandleFactory<T> writeHandleFactory) {
-    super(recordItr);
-    this.areRecordsSorted = areRecordsSorted;
-    this.hoodieConfig = config;
-    this.instantTime = instantTime;
-    this.hoodieTable = hoodieTable;
-    this.idPrefix = idPrefix;
-    this.sparkTaskContextSupplier = sparkTaskContextSupplier;
-    this.writeHandleFactory = writeHandleFactory;
-  }
-
-  // Used for caching HoodieRecord along with insertValue. We need this to offload computation work to buffering thread.
-  public static class HoodieInsertValueGenResult<T extends HoodieRecord> {
-    public T record;
-    public Option<IndexedRecord> insertValue;
-    // It caches the exception seen while fetching insert value.
-    public Option<Exception> exception = Option.empty();
-
-    public HoodieInsertValueGenResult(T record, Schema schema) {
-      this.record = record;
-      try {
-        this.insertValue = record.getData().getInsertValue(schema);
-      } catch (Exception e) {
-        this.exception = Option.of(e);
-      }
-    }
-  }
-
-  /**
-   * Transformer function to help transform a HoodieRecord. This transformer is used by BufferedIterator to offload some
-   * expensive operations of transformation to the reader thread.
-   */
-  static <T extends HoodieRecordPayload> Function<HoodieRecord<T>, HoodieInsertValueGenResult<HoodieRecord>> getTransformFunction(
-      Schema schema) {
-    return hoodieRecord -> new HoodieInsertValueGenResult(hoodieRecord, schema);
-  }
-
-  @Override
-  protected void start() {}
-
-  @Override
-  protected List<WriteStatus> computeNext() {
-    // Executor service used for launching writer thread.
-    BoundedInMemoryExecutor<HoodieRecord<T>, HoodieInsertValueGenResult<HoodieRecord>, List<WriteStatus>> bufferedIteratorExecutor =
-        null;
-    try {
-      final Schema schema = new Schema.Parser().parse(hoodieConfig.getSchema());
-      bufferedIteratorExecutor =
-          new SparkBoundedInMemoryExecutor<>(hoodieConfig, inputItr, getInsertHandler(), getTransformFunction(schema));
-      final List<WriteStatus> result = bufferedIteratorExecutor.execute();
-      assert result != null && !result.isEmpty() && !bufferedIteratorExecutor.isRemaining();
-      return result;
-    } catch (Exception e) {
-      throw new HoodieException(e);
-    } finally {
-      if (null != bufferedIteratorExecutor) {
-        bufferedIteratorExecutor.shutdownNow();
-      }
-    }
-  }
-
-  @Override
-  protected void end() {}
-
-  protected CopyOnWriteInsertHandler getInsertHandler() {
-    return new CopyOnWriteInsertHandler(hoodieConfig, instantTime, areRecordsSorted, hoodieTable, idPrefix,
-        sparkTaskContextSupplier, writeHandleFactory);
-  }
-}
diff --git a/hudi-client/src/main/java/org/apache/hudi/index/HoodieIndex.java b/hudi-client/src/main/java/org/apache/hudi/index/HoodieIndex.java
deleted file mode 100644
index 4043586..0000000
--- a/hudi-client/src/main/java/org/apache/hudi/index/HoodieIndex.java
+++ /dev/null
@@ -1,154 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hudi.index;
-
-import org.apache.hudi.ApiMaturityLevel;
-import org.apache.hudi.PublicAPIClass;
-import org.apache.hudi.PublicAPIMethod;
-import org.apache.hudi.client.WriteStatus;
-import org.apache.hudi.common.model.FileSlice;
-import org.apache.hudi.common.model.HoodieKey;
-import org.apache.hudi.common.model.HoodieRecord;
-import org.apache.hudi.common.model.HoodieRecordPayload;
-import org.apache.hudi.common.util.Option;
-import org.apache.hudi.common.util.ReflectionUtils;
-import org.apache.hudi.common.util.StringUtils;
-import org.apache.hudi.common.util.collection.Pair;
-import org.apache.hudi.config.HoodieWriteConfig;
-import org.apache.hudi.exception.HoodieIndexException;
-import org.apache.hudi.index.bloom.HoodieBloomIndex;
-import org.apache.hudi.index.bloom.HoodieGlobalBloomIndex;
-import org.apache.hudi.index.hbase.HBaseIndex;
-import org.apache.hudi.index.simple.HoodieGlobalSimpleIndex;
-import org.apache.hudi.index.simple.HoodieSimpleIndex;
-import org.apache.hudi.table.HoodieTable;
-
-import org.apache.spark.api.java.JavaPairRDD;
-import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
-
-import java.io.Serializable;
-
-/**
- * Base class for different types of indexes to determine the mapping from uuid.
- */
-@PublicAPIClass(maturity = ApiMaturityLevel.EVOLVING)
-public abstract class HoodieIndex<T extends HoodieRecordPayload> implements Serializable {
-
-  protected final HoodieWriteConfig config;
-
-  protected HoodieIndex(HoodieWriteConfig config) {
-    this.config = config;
-  }
-
-  public static <T extends HoodieRecordPayload> HoodieIndex<T> createIndex(
-      HoodieWriteConfig config) throws HoodieIndexException {
-    // first use index class config to create index.
-    if (!StringUtils.isNullOrEmpty(config.getIndexClass())) {
-      Object instance = ReflectionUtils.loadClass(config.getIndexClass(), config);
-      if (!(instance instanceof HoodieIndex)) {
-        throw new HoodieIndexException(config.getIndexClass() + " is not a subclass of HoodieIndex");
-      }
-      return (HoodieIndex) instance;
-    }
-    switch (config.getIndexType()) {
-      case HBASE:
-        return new HBaseIndex<>(config);
-      case INMEMORY:
-        return new InMemoryHashIndex<>(config);
-      case BLOOM:
-        return new HoodieBloomIndex<>(config);
-      case GLOBAL_BLOOM:
-        return new HoodieGlobalBloomIndex<>(config);
-      case SIMPLE:
-        return new HoodieSimpleIndex<>(config);
-      case GLOBAL_SIMPLE:
-        return new HoodieGlobalSimpleIndex<>(config);
-      default:
-        throw new HoodieIndexException("Index type unspecified, set " + config.getIndexType());
-    }
-  }
-
-  /**
-   * Checks if the given [Keys] exists in the hoodie table and returns [Key, Option[partitionPath, fileID]] If the
-   * optional is empty, then the key is not found.
-   */
-  @PublicAPIMethod(maturity = ApiMaturityLevel.STABLE)
-  public abstract JavaPairRDD<HoodieKey, Option<Pair<String, String>>> fetchRecordLocation(
-      JavaRDD<HoodieKey> hoodieKeys, final JavaSparkContext jsc, HoodieTable<T> hoodieTable);
-
-  /**
-   * Looks up the index and tags each incoming record with a location of a file that contains the row (if it is actually
-   * present).
-   */
-  @PublicAPIMethod(maturity = ApiMaturityLevel.STABLE)
-  public abstract JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, JavaSparkContext jsc,
-                                                       HoodieTable<T> hoodieTable) throws HoodieIndexException;
-
-  /**
-   * Extracts the location of written records, and updates the index.
-   * <p>
-   * TODO(vc): We may need to propagate the record as well in a WriteStatus class
-   */
-  @PublicAPIMethod(maturity = ApiMaturityLevel.STABLE)
-  public abstract JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, JavaSparkContext jsc,
-                                                      HoodieTable<T> hoodieTable) throws HoodieIndexException;
-
-  /**
-   * Rollback the efffects of the commit made at instantTime.
-   */
-  @PublicAPIMethod(maturity = ApiMaturityLevel.STABLE)
-  public abstract boolean rollbackCommit(String instantTime);
-
-  /**
-   * An index is `global` if {@link HoodieKey} to fileID mapping, does not depend on the `partitionPath`. Such an
-   * implementation is able to obtain the same mapping, for two hoodie keys with same `recordKey` but different
-   * `partitionPath`
-   *
-   * @return whether or not, the index implementation is global in nature
-   */
-  @PublicAPIMethod(maturity = ApiMaturityLevel.STABLE)
-  public abstract boolean isGlobal();
-
-  /**
-   * This is used by storage to determine, if its safe to send inserts, straight to the log, i.e having a
-   * {@link FileSlice}, with no data file.
-   *
-   * @return Returns true/false depending on whether the impl has this capability
-   */
-  @PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING)
-  public abstract boolean canIndexLogFiles();
-
-  /**
-   * An index is "implicit" with respect to storage, if just writing new data to a file slice, updates the index as
-   * well. This is used by storage, to save memory footprint in certain cases.
-   */
-  @PublicAPIMethod(maturity = ApiMaturityLevel.STABLE)
-  public abstract boolean isImplicitWithStorage();
-
-  /**
-   * Each index type should implement it's own logic to release any resources acquired during the process.
-   */
-  public void close() {
-  }
-
-  public enum IndexType {
-    HBASE, INMEMORY, BLOOM, GLOBAL_BLOOM, SIMPLE, GLOBAL_SIMPLE
-  }
-}
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/HoodieCopyOnWriteTable.java b/hudi-client/src/main/java/org/apache/hudi/table/HoodieCopyOnWriteTable.java
deleted file mode 100644
index a4bcbbf..0000000
--- a/hudi-client/src/main/java/org/apache/hudi/table/HoodieCopyOnWriteTable.java
+++ /dev/null
@@ -1,216 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hudi.table;
-
-import org.apache.hudi.avro.model.HoodieCleanMetadata;
-import org.apache.hudi.avro.model.HoodieCompactionPlan;
-import org.apache.hudi.avro.model.HoodieRestoreMetadata;
-import org.apache.hudi.avro.model.HoodieRollbackMetadata;
-import org.apache.hudi.avro.model.HoodieSavepointMetadata;
-import org.apache.hudi.client.WriteStatus;
-import org.apache.hudi.common.model.HoodieBaseFile;
-import org.apache.hudi.common.model.HoodieKey;
-import org.apache.hudi.common.model.HoodieRecord;
-import org.apache.hudi.common.model.HoodieRecordPayload;
-import org.apache.hudi.common.table.HoodieTableMetaClient;
-import org.apache.hudi.common.table.timeline.HoodieInstant;
-import org.apache.hudi.common.table.timeline.HoodieTimeline;
-import org.apache.hudi.common.util.Option;
-import org.apache.hudi.config.HoodieWriteConfig;
-import org.apache.hudi.exception.HoodieNotSupportedException;
-import org.apache.hudi.exception.HoodieUpsertException;
-import org.apache.hudi.io.HoodieCreateHandle;
-import org.apache.hudi.io.HoodieMergeHandle;
-import org.apache.hudi.io.HoodieSortedMergeHandle;
-import org.apache.hudi.table.action.HoodieWriteMetadata;
-import org.apache.hudi.table.action.bootstrap.BootstrapCommitActionExecutor;
-import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata;
-import org.apache.hudi.table.action.clean.CleanActionExecutor;
-import org.apache.hudi.table.action.commit.BulkInsertCommitActionExecutor;
-import org.apache.hudi.table.action.commit.BulkInsertPreppedCommitActionExecutor;
-import org.apache.hudi.table.action.commit.DeleteCommitActionExecutor;
-import org.apache.hudi.table.action.commit.InsertCommitActionExecutor;
-import org.apache.hudi.table.action.commit.InsertOverwriteCommitActionExecutor;
-import org.apache.hudi.table.action.commit.InsertPreppedCommitActionExecutor;
-import org.apache.hudi.table.action.commit.MergeHelper;
-import org.apache.hudi.table.action.commit.UpsertCommitActionExecutor;
-import org.apache.hudi.table.action.commit.UpsertPreppedCommitActionExecutor;
-import org.apache.hudi.table.action.restore.CopyOnWriteRestoreActionExecutor;
-import org.apache.hudi.table.action.rollback.CopyOnWriteRollbackActionExecutor;
-import org.apache.hudi.table.action.savepoint.SavepointActionExecutor;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.log4j.LogManager;
-import org.apache.log4j.Logger;
-import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
-
-import java.io.IOException;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-
-/**
- * Implementation of a very heavily read-optimized Hoodie Table where, all data is stored in base files, with
- * zero read amplification.
- * <p>
- * INSERTS - Produce new files, block aligned to desired size (or) Merge with the smallest existing file, to expand it
- * <p>
- * UPDATES - Produce a new version of the file, just replacing the updated records with new values
- */
-public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends HoodieTable<T> {
-
-  private static final Logger LOG = LogManager.getLogger(HoodieCopyOnWriteTable.class);
-
-  public HoodieCopyOnWriteTable(HoodieWriteConfig config, Configuration hadoopConf, HoodieTableMetaClient metaClient) {
-    super(config, hadoopConf, metaClient);
-  }
-
-  @Override
-  public HoodieWriteMetadata upsert(JavaSparkContext jsc, String instantTime, JavaRDD<HoodieRecord<T>> records) {
-    return new UpsertCommitActionExecutor<>(jsc, config, this, instantTime, records).execute();
-  }
-
-  @Override
-  public HoodieWriteMetadata insert(JavaSparkContext jsc, String instantTime, JavaRDD<HoodieRecord<T>> records) {
-    return new InsertCommitActionExecutor<>(jsc, config, this, instantTime, records).execute();
-  }
-
-  @Override
-  public HoodieWriteMetadata bulkInsert(JavaSparkContext jsc, String instantTime, JavaRDD<HoodieRecord<T>> records,
-      Option<BulkInsertPartitioner> userDefinedBulkInsertPartitioner) {
-    return new BulkInsertCommitActionExecutor(jsc, config,
-        this, instantTime, records, userDefinedBulkInsertPartitioner).execute();
-  }
-
-  @Override
-  public HoodieWriteMetadata delete(JavaSparkContext jsc, String instantTime, JavaRDD<HoodieKey> keys) {
-    return new DeleteCommitActionExecutor<>(jsc, config, this, instantTime, keys).execute();
-  }
-
-  @Override
-  public HoodieWriteMetadata upsertPrepped(JavaSparkContext jsc, String instantTime,
-      JavaRDD<HoodieRecord<T>> preppedRecords) {
-    return new UpsertPreppedCommitActionExecutor<>(jsc, config, this, instantTime, preppedRecords).execute();
-  }
-
-  @Override
-  public HoodieWriteMetadata insertPrepped(JavaSparkContext jsc, String instantTime,
-      JavaRDD<HoodieRecord<T>> preppedRecords) {
-    return new InsertPreppedCommitActionExecutor<>(jsc, config, this, instantTime, preppedRecords).execute();
-  }
-
-  @Override
-  public HoodieWriteMetadata bulkInsertPrepped(JavaSparkContext jsc, String instantTime,
-      JavaRDD<HoodieRecord<T>> preppedRecords,  Option<BulkInsertPartitioner> userDefinedBulkInsertPartitioner) {
-    return new BulkInsertPreppedCommitActionExecutor(jsc, config,
-        this, instantTime, preppedRecords, userDefinedBulkInsertPartitioner).execute();
-  }
-
-  @Override
-  public HoodieWriteMetadata insertOverwrite(JavaSparkContext jsc, String instantTime,
-                                                      JavaRDD<HoodieRecord<T>> records) {
-    return new InsertOverwriteCommitActionExecutor<>(jsc, config, this, instantTime, records).execute();
-  }
-
-  @Override
-  public Option<HoodieCompactionPlan> scheduleCompaction(JavaSparkContext jsc, String instantTime, Option<Map<String, String>> extraMetadata) {
-    throw new HoodieNotSupportedException("Compaction is not supported on a CopyOnWrite table");
-  }
-
-  @Override
-  public HoodieWriteMetadata compact(JavaSparkContext jsc, String compactionInstantTime) {
-    throw new HoodieNotSupportedException("Compaction is not supported on a CopyOnWrite table");
-  }
-
-  @Override
-  public HoodieBootstrapWriteMetadata bootstrap(JavaSparkContext jsc, Option<Map<String, String>> extraMetadata) {
-    return new BootstrapCommitActionExecutor(jsc, config, this, extraMetadata).execute();
-  }
-
-  @Override
-  public void rollbackBootstrap(JavaSparkContext jsc, String instantTime) {
-    new CopyOnWriteRestoreActionExecutor(jsc, config, this, instantTime, HoodieTimeline.INIT_INSTANT_TS).execute();
-  }
-
-  public Iterator<List<WriteStatus>> handleUpdate(String instantTime, String partitionPath, String fileId,
-      Map<String, HoodieRecord<T>> keyToNewRecords, HoodieBaseFile oldDataFile) throws IOException {
-    // these are updates
-    HoodieMergeHandle upsertHandle = getUpdateHandle(instantTime, partitionPath, fileId, keyToNewRecords, oldDataFile);
-    return handleUpdateInternal(upsertHandle, instantTime, fileId);
-  }
-
-  protected Iterator<List<WriteStatus>> handleUpdateInternal(HoodieMergeHandle upsertHandle, String instantTime,
-      String fileId) throws IOException {
-    if (upsertHandle.getOldFilePath() == null) {
-      throw new HoodieUpsertException(
-          "Error in finding the old file path at commit " + instantTime + " for fileId: " + fileId);
-    } else {
-      MergeHelper.runMerge(this, upsertHandle);
-    }
-
-    // TODO(vc): This needs to be revisited
-    if (upsertHandle.getWriteStatus().getPartitionPath() == null) {
-      LOG.info("Upsert Handle has partition path as null " + upsertHandle.getOldFilePath() + ", "
-          + upsertHandle.getWriteStatus());
-    }
-    return Collections.singletonList(Collections.singletonList(upsertHandle.getWriteStatus())).iterator();
-  }
-
-  protected HoodieMergeHandle getUpdateHandle(String instantTime, String partitionPath, String fileId,
-      Map<String, HoodieRecord<T>> keyToNewRecords, HoodieBaseFile dataFileToBeMerged) {
-    if (requireSortedRecords()) {
-      return new HoodieSortedMergeHandle<>(config, instantTime, this, keyToNewRecords, partitionPath, fileId,
-          dataFileToBeMerged, sparkTaskContextSupplier);
-    } else {
-      return new HoodieMergeHandle<>(config, instantTime, this, keyToNewRecords, partitionPath, fileId,
-          dataFileToBeMerged, sparkTaskContextSupplier);
-    }
-  }
-
-  public Iterator<List<WriteStatus>> handleInsert(String instantTime, String partitionPath, String fileId,
-      Map<String, HoodieRecord<? extends HoodieRecordPayload>> recordMap) {
-    HoodieCreateHandle createHandle =
-        new HoodieCreateHandle(config, instantTime, this, partitionPath, fileId, recordMap, sparkTaskContextSupplier);
-    createHandle.write();
-    return Collections.singletonList(Collections.singletonList(createHandle.close())).iterator();
-  }
-
-  @Override
-  public HoodieCleanMetadata clean(JavaSparkContext jsc, String cleanInstantTime) {
-    return new CleanActionExecutor(jsc, config, this, cleanInstantTime).execute();
-  }
-
-  @Override
-  public HoodieRollbackMetadata rollback(JavaSparkContext jsc, String rollbackInstantTime, HoodieInstant commitInstant, boolean deleteInstants) {
-    return new CopyOnWriteRollbackActionExecutor(jsc, config, this, rollbackInstantTime, commitInstant, deleteInstants).execute();
-  }
-
-  @Override
-  public HoodieSavepointMetadata savepoint(JavaSparkContext jsc, String instantToSavepoint, String user, String comment) {
-    return new SavepointActionExecutor(jsc, config, this, instantToSavepoint, user, comment).execute();
-  }
-
-  @Override
-  public HoodieRestoreMetadata restore(JavaSparkContext jsc, String restoreInstantTime, String instantToRestore) {
-    return new CopyOnWriteRestoreActionExecutor(jsc, config, this, restoreInstantTime, instantToRestore).execute();
-  }
-
-}
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/HoodieMergeOnReadTable.java b/hudi-client/src/main/java/org/apache/hudi/table/HoodieMergeOnReadTable.java
deleted file mode 100644
index a236cdb..0000000
--- a/hudi-client/src/main/java/org/apache/hudi/table/HoodieMergeOnReadTable.java
+++ /dev/null
@@ -1,161 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hudi.table;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hudi.avro.model.HoodieCompactionPlan;
-import org.apache.hudi.avro.model.HoodieRestoreMetadata;
-import org.apache.hudi.avro.model.HoodieRollbackMetadata;
-import org.apache.hudi.common.model.HoodieKey;
-import org.apache.hudi.common.model.HoodieRecord;
-import org.apache.hudi.common.model.HoodieRecordPayload;
-import org.apache.hudi.common.model.HoodieWriteStat;
-import org.apache.hudi.common.table.HoodieTableMetaClient;
-import org.apache.hudi.common.table.timeline.HoodieInstant;
-
-import org.apache.hudi.common.table.timeline.HoodieTimeline;
-import org.apache.hudi.common.util.Option;
-import org.apache.hudi.config.HoodieWriteConfig;
-import org.apache.hudi.exception.HoodieIOException;
-import org.apache.hudi.table.action.HoodieWriteMetadata;
-import org.apache.hudi.table.action.bootstrap.BootstrapDeltaCommitActionExecutor;
-import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata;
-import org.apache.hudi.table.action.compact.RunCompactionActionExecutor;
-import org.apache.hudi.table.action.deltacommit.BulkInsertDeltaCommitActionExecutor;
-import org.apache.hudi.table.action.deltacommit.BulkInsertPreppedDeltaCommitActionExecutor;
-import org.apache.hudi.table.action.deltacommit.DeleteDeltaCommitActionExecutor;
-import org.apache.hudi.table.action.deltacommit.InsertDeltaCommitActionExecutor;
-import org.apache.hudi.table.action.deltacommit.InsertPreppedDeltaCommitActionExecutor;
-import org.apache.hudi.table.action.deltacommit.UpsertDeltaCommitActionExecutor;
-import org.apache.hudi.table.action.deltacommit.UpsertPreppedDeltaCommitActionExecutor;
-import org.apache.hudi.table.action.compact.ScheduleCompactionActionExecutor;
-import org.apache.hudi.table.action.restore.MergeOnReadRestoreActionExecutor;
-import org.apache.hudi.table.action.rollback.MergeOnReadRollbackActionExecutor;
-import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
-
-import java.util.List;
-import java.util.Map;
-
-/**
- * Implementation of a more real-time Hoodie Table the provides tradeoffs on read and write cost/amplification.
- *
- * <p>
- * INSERTS - Same as HoodieCopyOnWriteTable - Produce new files, block aligned to desired size (or) Merge with the
- * smallest existing file, to expand it
- * </p>
- * <p>
- * UPDATES - Appends the changes to a rolling log file maintained per file Id. Compaction merges the log file into the
- * base file.
- * </p>
- * <p>
- * WARNING - MOR table type does not support nested rollbacks, every rollback must be followed by an attempted commit
- * action
- * </p>
- */
-public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends HoodieCopyOnWriteTable<T> {
-
-  HoodieMergeOnReadTable(HoodieWriteConfig config, Configuration hadoopConf, HoodieTableMetaClient metaClient) {
-    super(config, hadoopConf, metaClient);
-  }
-
-  @Override
-  public HoodieWriteMetadata upsert(JavaSparkContext jsc, String instantTime, JavaRDD<HoodieRecord<T>> records) {
-    return new UpsertDeltaCommitActionExecutor<>(jsc, config, this, instantTime, records).execute();
-  }
-
-  @Override
-  public HoodieWriteMetadata insert(JavaSparkContext jsc, String instantTime, JavaRDD<HoodieRecord<T>> records) {
-    return new InsertDeltaCommitActionExecutor<>(jsc, config, this, instantTime, records).execute();
-  }
-
-  @Override
-  public HoodieWriteMetadata bulkInsert(JavaSparkContext jsc, String instantTime, JavaRDD<HoodieRecord<T>> records,
-      Option<BulkInsertPartitioner> userDefinedBulkInsertPartitioner) {
-    return new BulkInsertDeltaCommitActionExecutor(jsc, config,
-        this, instantTime, records, userDefinedBulkInsertPartitioner).execute();
-  }
-
-  @Override
-  public HoodieWriteMetadata delete(JavaSparkContext jsc, String instantTime, JavaRDD<HoodieKey> keys) {
-    return new DeleteDeltaCommitActionExecutor<>(jsc, config, this, instantTime, keys).execute();
-  }
-
-  @Override
-  public HoodieWriteMetadata upsertPrepped(JavaSparkContext jsc, String instantTime,
-      JavaRDD<HoodieRecord<T>> preppedRecords) {
-    return new UpsertPreppedDeltaCommitActionExecutor<>(jsc, config, this, instantTime, preppedRecords).execute();
-  }
-
-  @Override
-  public HoodieWriteMetadata insertPrepped(JavaSparkContext jsc, String instantTime,
-      JavaRDD<HoodieRecord<T>> preppedRecords) {
-    return new InsertPreppedDeltaCommitActionExecutor<>(jsc, config, this, instantTime, preppedRecords).execute();
-  }
-
-  @Override
-  public HoodieWriteMetadata bulkInsertPrepped(JavaSparkContext jsc, String instantTime,
-      JavaRDD<HoodieRecord<T>> preppedRecords,  Option<BulkInsertPartitioner> userDefinedBulkInsertPartitioner) {
-    return new BulkInsertPreppedDeltaCommitActionExecutor(jsc, config,
-        this, instantTime, preppedRecords, userDefinedBulkInsertPartitioner).execute();
-  }
-
-  @Override
-  public Option<HoodieCompactionPlan> scheduleCompaction(JavaSparkContext jsc, String instantTime, Option<Map<String, String>> extraMetadata) {
-    ScheduleCompactionActionExecutor scheduleCompactionExecutor = new ScheduleCompactionActionExecutor(
-        jsc, config, this, instantTime, extraMetadata);
-    return scheduleCompactionExecutor.execute();
-  }
-
-  @Override
-  public HoodieWriteMetadata compact(JavaSparkContext jsc, String compactionInstantTime) {
-    RunCompactionActionExecutor compactionExecutor = new RunCompactionActionExecutor(jsc, config, this, compactionInstantTime);
-    return compactionExecutor.execute();
-  }
-
-  @Override
-  public HoodieBootstrapWriteMetadata bootstrap(JavaSparkContext jsc, Option<Map<String, String>> extraMetadata) {
-    return new BootstrapDeltaCommitActionExecutor(jsc, config, this, extraMetadata).execute();
-  }
-
-  @Override
-  public void rollbackBootstrap(JavaSparkContext jsc, String instantTime) {
-    new MergeOnReadRestoreActionExecutor(jsc, config, this, instantTime, HoodieTimeline.INIT_INSTANT_TS).execute();
-  }
-
-  @Override
-  public HoodieRollbackMetadata rollback(JavaSparkContext jsc,
-                                         String rollbackInstantTime,
-                                         HoodieInstant commitInstant,
-                                         boolean deleteInstants) {
-    return new MergeOnReadRollbackActionExecutor(jsc, config, this, rollbackInstantTime, commitInstant, deleteInstants).execute();
-  }
-
-  @Override
-  public HoodieRestoreMetadata restore(JavaSparkContext jsc, String restoreInstantTime, String instantToRestore) {
-    return new MergeOnReadRestoreActionExecutor(jsc, config, this, restoreInstantTime, instantToRestore).execute();
-  }
-
-  @Override
-  public void finalizeWrite(JavaSparkContext jsc, String instantTs, List<HoodieWriteStat> stats)
-      throws HoodieIOException {
-    // delegate to base class for MOR tables
-    super.finalizeWrite(jsc, instantTs, stats);
-  }
-}
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/WorkloadProfile.java b/hudi-client/src/main/java/org/apache/hudi/table/WorkloadProfile.java
deleted file mode 100644
index 8179c90..0000000
--- a/hudi-client/src/main/java/org/apache/hudi/table/WorkloadProfile.java
+++ /dev/null
@@ -1,121 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hudi.table;
-
-import org.apache.hudi.common.model.HoodieRecord;
-import org.apache.hudi.common.model.HoodieRecordLocation;
-import org.apache.hudi.common.model.HoodieRecordPayload;
-import org.apache.hudi.common.util.Option;
-
-import org.apache.spark.api.java.JavaRDD;
-
-import java.io.Serializable;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Set;
-
-import scala.Tuple2;
-
-/**
- * Information about incoming records for upsert/insert obtained either via sampling or introspecting the data fully.
- * <p>
- * TODO(vc): Think about obtaining this directly from index.tagLocation
- */
-public class WorkloadProfile<T extends HoodieRecordPayload> implements Serializable {
-
-  /**
-   * Input workload.
-   */
-  private final JavaRDD<HoodieRecord<T>> taggedRecords;
-
-  /**
-   * Computed workload profile.
-   */
-  private final HashMap<String, WorkloadStat> partitionPathStatMap;
-
-  /**
-   * Global workloadStat.
-   */
-  private final WorkloadStat globalStat;
-
-  public WorkloadProfile(JavaRDD<HoodieRecord<T>> taggedRecords) {
-    this.taggedRecords = taggedRecords;
-    this.partitionPathStatMap = new HashMap<>();
-    this.globalStat = new WorkloadStat();
-    buildProfile();
-  }
-
-  /**
-   *  Method help to build WorkloadProfile.
-   */
-  private void buildProfile() {
-    // group the records by partitionPath + currentLocation combination, count the number of
-    // records in each partition
-    Map<Tuple2<String, Option<HoodieRecordLocation>>, Long> partitionLocationCounts = taggedRecords
-        .mapToPair(record -> new Tuple2<>(
-            new Tuple2<>(record.getPartitionPath(), Option.ofNullable(record.getCurrentLocation())), record))
-        .countByKey();
-
-    // count the number of both inserts and updates in each partition, update the counts to workLoadStats
-    for (Map.Entry<Tuple2<String, Option<HoodieRecordLocation>>, Long> e : partitionLocationCounts.entrySet()) {
-      String partitionPath = e.getKey()._1();
-      Long count = e.getValue();
-      Option<HoodieRecordLocation> locOption = e.getKey()._2();
-
-      if (!partitionPathStatMap.containsKey(partitionPath)) {
-        partitionPathStatMap.put(partitionPath, new WorkloadStat());
-      }
-
-      if (locOption.isPresent()) {
-        // update
-        partitionPathStatMap.get(partitionPath).addUpdates(locOption.get(), count);
-        globalStat.addUpdates(locOption.get(), count);
-      } else {
-        // insert
-        partitionPathStatMap.get(partitionPath).addInserts(count);
-        globalStat.addInserts(count);
-      }
-    }
-  }
-
-  public WorkloadStat getGlobalStat() {
-    return globalStat;
-  }
-
-  public Set<String> getPartitionPaths() {
-    return partitionPathStatMap.keySet();
-  }
-
-  public HashMap<String, WorkloadStat> getPartitionPathStatMap() {
-    return partitionPathStatMap;
-  }
-
-  public WorkloadStat getWorkloadStat(String partitionPath) {
-    return partitionPathStatMap.get(partitionPath);
-  }
-
-  @Override
-  public String toString() {
-    final StringBuilder sb = new StringBuilder("WorkloadProfile {");
-    sb.append("globalStat=").append(globalStat).append(", ");
-    sb.append("partitionStat=").append(partitionPathStatMap);
-    sb.append('}');
-    return sb.toString();
-  }
-}
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapDeltaCommitActionExecutor.java b/hudi-client/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapDeltaCommitActionExecutor.java
deleted file mode 100644
index 08760cc..0000000
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/bootstrap/BootstrapDeltaCommitActionExecutor.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hudi.table.action.bootstrap;
-
-import java.util.Map;
-import org.apache.hudi.common.model.HoodieRecord;
-import org.apache.hudi.common.model.HoodieRecordPayload;
-import org.apache.hudi.common.table.timeline.HoodieTimeline;
-import org.apache.hudi.common.util.Option;
-import org.apache.hudi.config.HoodieWriteConfig;
-import org.apache.hudi.table.HoodieTable;
-import org.apache.hudi.table.action.commit.CommitActionExecutor;
-import org.apache.hudi.table.action.deltacommit.BulkInsertDeltaCommitActionExecutor;
-import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
-
-public class BootstrapDeltaCommitActionExecutor<T extends HoodieRecordPayload<T>>
-    extends BootstrapCommitActionExecutor<T> {
-
-  public BootstrapDeltaCommitActionExecutor(JavaSparkContext jsc,
-      HoodieWriteConfig config, HoodieTable<?> table,
-      Option<Map<String, String>> extraMetadata) {
-    super(jsc, config, table, extraMetadata);
-  }
-
-  protected CommitActionExecutor<T> getBulkInsertActionExecutor(JavaRDD<HoodieRecord> inputRecordsRDD) {
-    return new BulkInsertDeltaCommitActionExecutor(jsc, new HoodieWriteConfig.Builder().withProps(config.getProps())
-        .withSchema(bootstrapSchema).build(), table, HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS,
-        inputRecordsRDD, extraMetadata);
-  }
-}
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/clean/CleanActionExecutor.java b/hudi-client/src/main/java/org/apache/hudi/table/action/clean/CleanActionExecutor.java
deleted file mode 100644
index 5261447..0000000
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/clean/CleanActionExecutor.java
+++ /dev/null
@@ -1,296 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hudi.table.action.clean;
-
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hudi.avro.model.HoodieActionInstant;
-import org.apache.hudi.avro.model.HoodieCleanMetadata;
-import org.apache.hudi.avro.model.HoodieCleanerPlan;
-import org.apache.hudi.avro.model.HoodieCleanFileInfo;
-import org.apache.hudi.common.HoodieCleanStat;
-import org.apache.hudi.common.model.CleanFileInfo;
-import org.apache.hudi.common.model.HoodieCleaningPolicy;
-import org.apache.hudi.common.table.timeline.HoodieInstant;
-import org.apache.hudi.common.table.timeline.HoodieTimeline;
-import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
-import org.apache.hudi.common.util.CleanerUtils;
-import org.apache.hudi.common.util.CollectionUtils;
-import org.apache.hudi.common.util.HoodieTimer;
-import org.apache.hudi.common.util.Option;
-import org.apache.hudi.common.util.ValidationUtils;
-import org.apache.hudi.common.util.collection.Pair;
-import org.apache.hudi.config.HoodieWriteConfig;
-import org.apache.hudi.exception.HoodieIOException;
-import org.apache.hudi.table.HoodieTable;
-import org.apache.hudi.table.action.BaseActionExecutor;
-import org.apache.log4j.LogManager;
-import org.apache.log4j.Logger;
-import org.apache.spark.api.java.JavaSparkContext;
-import org.apache.spark.api.java.function.PairFlatMapFunction;
-import scala.Tuple2;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.stream.Collectors;
-
-public class CleanActionExecutor extends BaseActionExecutor<HoodieCleanMetadata> {
-
-  private static final long serialVersionUID = 1L;
-  private static final Logger LOG = LogManager.getLogger(CleanActionExecutor.class);
-
-  public CleanActionExecutor(JavaSparkContext jsc, HoodieWriteConfig config, HoodieTable<?> table, String instantTime) {
-    super(jsc, config, table, instantTime);
-  }
-
-  /**
-   * Generates List of files to be cleaned.
-   *
-   * @param jsc JavaSparkContext
-   * @return Cleaner Plan
-   */
-  HoodieCleanerPlan requestClean(JavaSparkContext jsc) {
-    try {
-      CleanPlanner<?> planner = new CleanPlanner<>(table, config);
-      Option<HoodieInstant> earliestInstant = planner.getEarliestCommitToRetain();
-      List<String> partitionsToClean = planner.getPartitionPathsToClean(earliestInstant);
-
-      if (partitionsToClean.isEmpty()) {
-        LOG.info("Nothing to clean here. It is already clean");
-        return HoodieCleanerPlan.newBuilder().setPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS.name()).build();
-      }
-      LOG.info("Total Partitions to clean : " + partitionsToClean.size() + ", with policy " + config.getCleanerPolicy());
-      int cleanerParallelism = Math.min(partitionsToClean.size(), config.getCleanerParallelism());
-      LOG.info("Using cleanerParallelism: " + cleanerParallelism);
-
-      jsc.setJobGroup(this.getClass().getSimpleName(), "Generates list of file slices to be cleaned");
-      Map<String, List<HoodieCleanFileInfo>> cleanOps = jsc
-          .parallelize(partitionsToClean, cleanerParallelism)
-          .map(partitionPathToClean -> Pair.of(partitionPathToClean, planner.getDeletePaths(partitionPathToClean)))
-          .collect().stream()
-          .collect(Collectors.toMap(Pair::getKey, y -> CleanerUtils.convertToHoodieCleanFileInfoList(y.getValue())));
-
-      return new HoodieCleanerPlan(earliestInstant
-          .map(x -> new HoodieActionInstant(x.getTimestamp(), x.getAction(), x.getState().name())).orElse(null),
-          config.getCleanerPolicy().name(), CollectionUtils.createImmutableMap(),
-          CleanPlanner.LATEST_CLEAN_PLAN_VERSION, cleanOps);
-    } catch (IOException e) {
-      throw new HoodieIOException("Failed to schedule clean operation", e);
-    }
-  }
-
-  private static PairFlatMapFunction<Iterator<Tuple2<String, CleanFileInfo>>, String, PartitionCleanStat>
-        deleteFilesFunc(HoodieTable table) {
-    return (PairFlatMapFunction<Iterator<Tuple2<String, CleanFileInfo>>, String, PartitionCleanStat>) iter -> {
-      Map<String, PartitionCleanStat> partitionCleanStatMap = new HashMap<>();
-      FileSystem fs = table.getMetaClient().getFs();
-      while (iter.hasNext()) {
-        Tuple2<String, CleanFileInfo> partitionDelFileTuple = iter.next();
-        String partitionPath = partitionDelFileTuple._1();
-        Path deletePath = new Path(partitionDelFileTuple._2().getFilePath());
-        String deletePathStr = deletePath.toString();
-        Boolean deletedFileResult = deleteFileAndGetResult(fs, deletePathStr);
-        if (!partitionCleanStatMap.containsKey(partitionPath)) {
-          partitionCleanStatMap.put(partitionPath, new PartitionCleanStat(partitionPath));
-        }
-        boolean isBootstrapBasePathFile = partitionDelFileTuple._2().isBootstrapBaseFile();
-        PartitionCleanStat partitionCleanStat = partitionCleanStatMap.get(partitionPath);
-        if (isBootstrapBasePathFile) {
-          // For Bootstrap Base file deletions, store the full file path.
-          partitionCleanStat.addDeleteFilePatterns(deletePath.toString(), true);
-          partitionCleanStat.addDeletedFileResult(deletePath.toString(), deletedFileResult, true);
-        } else {
-          partitionCleanStat.addDeleteFilePatterns(deletePath.getName(), false);
-          partitionCleanStat.addDeletedFileResult(deletePath.getName(), deletedFileResult, false);
-        }
-      }
-      return partitionCleanStatMap.entrySet().stream().map(e -> new Tuple2<>(e.getKey(), e.getValue()))
-          .collect(Collectors.toList()).iterator();
-    };
-  }
-
-  private static Boolean deleteFileAndGetResult(FileSystem fs, String deletePathStr) throws IOException {
-    Path deletePath = new Path(deletePathStr);
-    LOG.debug("Working on delete path :" + deletePath);
-    try {
-      boolean deleteResult = fs.delete(deletePath, false);
-      if (deleteResult) {
-        LOG.debug("Cleaned file at path :" + deletePath);
-      }
-      return deleteResult;
-    } catch (FileNotFoundException fio) {
-      // With cleanPlan being used for retried cleaning operations, its possible to clean a file twice
-      return false;
-    }
-  }
-
-  /**
-   * Performs cleaning of partition paths according to cleaning policy and returns the number of files cleaned. Handles
-   * skews in partitions to clean by making files to clean as the unit of task distribution.
-   *
-   * @throws IllegalArgumentException if unknown cleaning policy is provided
-   */
-  List<HoodieCleanStat> clean(JavaSparkContext jsc, HoodieCleanerPlan cleanerPlan) {
-    int cleanerParallelism = Math.min(
-        (int) (cleanerPlan.getFilePathsToBeDeletedPerPartition().values().stream().mapToInt(List::size).count()),
-        config.getCleanerParallelism());
-    LOG.info("Using cleanerParallelism: " + cleanerParallelism);
-
-    jsc.setJobGroup(this.getClass().getSimpleName(), "Perform cleaning of partitions");
-    List<Tuple2<String, PartitionCleanStat>> partitionCleanStats = jsc
-        .parallelize(cleanerPlan.getFilePathsToBeDeletedPerPartition().entrySet().stream()
-            .flatMap(x -> x.getValue().stream().map(y -> new Tuple2<>(x.getKey(),
-              new CleanFileInfo(y.getFilePath(), y.getIsBootstrapBaseFile()))))
-            .collect(Collectors.toList()), cleanerParallelism)
-        .mapPartitionsToPair(deleteFilesFunc(table))
-        .reduceByKey(PartitionCleanStat::merge).collect();
-
-    Map<String, PartitionCleanStat> partitionCleanStatsMap = partitionCleanStats.stream()
-        .collect(Collectors.toMap(Tuple2::_1, Tuple2::_2));
-
-    // Return PartitionCleanStat for each partition passed.
-    return cleanerPlan.getFilePathsToBeDeletedPerPartition().keySet().stream().map(partitionPath -> {
-      PartitionCleanStat partitionCleanStat = partitionCleanStatsMap.containsKey(partitionPath)
-          ? partitionCleanStatsMap.get(partitionPath)
-          : new PartitionCleanStat(partitionPath);
-      HoodieActionInstant actionInstant = cleanerPlan.getEarliestInstantToRetain();
-      return HoodieCleanStat.newBuilder().withPolicy(config.getCleanerPolicy()).withPartitionPath(partitionPath)
-          .withEarliestCommitRetained(Option.ofNullable(
-              actionInstant != null
-                  ? new HoodieInstant(HoodieInstant.State.valueOf(actionInstant.getState()),
-                  actionInstant.getAction(), actionInstant.getTimestamp())
-                  : null))
-          .withDeletePathPattern(partitionCleanStat.deletePathPatterns())
-          .withSuccessfulDeletes(partitionCleanStat.successDeleteFiles())
-          .withFailedDeletes(partitionCleanStat.failedDeleteFiles())
-          .withDeleteBootstrapBasePathPatterns(partitionCleanStat.getDeleteBootstrapBasePathPatterns())
-          .withSuccessfulDeleteBootstrapBaseFiles(partitionCleanStat.getSuccessfulDeleteBootstrapBaseFiles())
-          .withFailedDeleteBootstrapBaseFiles(partitionCleanStat.getFailedDeleteBootstrapBaseFiles())
-          .build();
-    }).collect(Collectors.toList());
-  }
-
-  /**
-   * Creates a Cleaner plan if there are files to be cleaned and stores them in instant file.
-   * Cleaner Plan contains absolute file paths.
-   *
-   * @param startCleanTime Cleaner Instant Time
-   * @return Cleaner Plan if generated
-   */
-  Option<HoodieCleanerPlan> requestClean(String startCleanTime) {
-    final HoodieCleanerPlan cleanerPlan = requestClean(jsc);
-    if ((cleanerPlan.getFilePathsToBeDeletedPerPartition() != null)
-        && !cleanerPlan.getFilePathsToBeDeletedPerPartition().isEmpty()
-        && cleanerPlan.getFilePathsToBeDeletedPerPartition().values().stream().mapToInt(List::size).sum() > 0) {
-      // Only create cleaner plan which does some work
-      final HoodieInstant cleanInstant = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.CLEAN_ACTION, startCleanTime);
-      // Save to both aux and timeline folder
-      try {
-        table.getActiveTimeline().saveToCleanRequested(cleanInstant, TimelineMetadataUtils.serializeCleanerPlan(cleanerPlan));
-        LOG.info("Requesting Cleaning with instant time " + cleanInstant);
-      } catch (IOException e) {
-        LOG.error("Got exception when saving cleaner requested file", e);
-        throw new HoodieIOException(e.getMessage(), e);
-      }
-      return Option.of(cleanerPlan);
-    }
-    return Option.empty();
-  }
-
-  /**
-   * Executes the Cleaner plan stored in the instant metadata.
-   */
-  void runPendingClean(HoodieTable<?> table, HoodieInstant cleanInstant) {
-    try {
-      HoodieCleanerPlan cleanerPlan = CleanerUtils.getCleanerPlan(table.getMetaClient(), cleanInstant);
-      runClean(table, cleanInstant, cleanerPlan);
-    } catch (IOException e) {
-      throw new HoodieIOException(e.getMessage(), e);
-    }
-  }
-
-  private HoodieCleanMetadata runClean(HoodieTable<?> table, HoodieInstant cleanInstant, HoodieCleanerPlan cleanerPlan) {
-    ValidationUtils.checkArgument(cleanInstant.getState().equals(HoodieInstant.State.REQUESTED)
-        || cleanInstant.getState().equals(HoodieInstant.State.INFLIGHT));
-
-    try {
-      final HoodieInstant inflightInstant;
-      final HoodieTimer timer = new HoodieTimer();
-      timer.startTimer();
-      if (cleanInstant.isRequested()) {
-        inflightInstant = table.getActiveTimeline().transitionCleanRequestedToInflight(cleanInstant,
-            TimelineMetadataUtils.serializeCleanerPlan(cleanerPlan));
-      } else {
-        inflightInstant = cleanInstant;
-      }
-
-      List<HoodieCleanStat> cleanStats = clean(jsc, cleanerPlan);
-      if (cleanStats.isEmpty()) {
-        return HoodieCleanMetadata.newBuilder().build();
-      }
-
-      table.getMetaClient().reloadActiveTimeline();
-      HoodieCleanMetadata metadata = CleanerUtils.convertCleanMetadata(
-          inflightInstant.getTimestamp(),
-          Option.of(timer.endTimer()),
-          cleanStats
-      );
-
-      table.getActiveTimeline().transitionCleanInflightToComplete(inflightInstant,
-          TimelineMetadataUtils.serializeCleanMetadata(metadata));
-      LOG.info("Marked clean started on " + inflightInstant.getTimestamp() + " as complete");
-      return metadata;
-    } catch (IOException e) {
-      throw new HoodieIOException("Failed to clean up after commit", e);
-    }
-  }
-
-  @Override
-  public HoodieCleanMetadata execute() {
-    // If there are inflight(failed) or previously requested clean operation, first perform them
-    List<HoodieInstant> pendingCleanInstants = table.getCleanTimeline()
-        .filterInflightsAndRequested().getInstants().collect(Collectors.toList());
-    if (pendingCleanInstants.size() > 0) {
-      pendingCleanInstants.forEach(hoodieInstant -> {
-        LOG.info("Finishing previously unfinished cleaner instant=" + hoodieInstant);
-        try {
-          runPendingClean(table, hoodieInstant);
-        } catch (Exception e) {
-          LOG.warn("Failed to perform previous clean operation, instant: " + hoodieInstant, e);
-        }
-      });
-      table.getMetaClient().reloadActiveTimeline();
-    }
-
-    // Plan and execute a new clean action
-    Option<HoodieCleanerPlan> cleanerPlanOpt = requestClean(instantTime);
-    if (cleanerPlanOpt.isPresent()) {
-      table.getMetaClient().reloadActiveTimeline();
-      HoodieCleanerPlan cleanerPlan = cleanerPlanOpt.get();
-      if ((cleanerPlan.getFilePathsToBeDeletedPerPartition() != null) && !cleanerPlan.getFilePathsToBeDeletedPerPartition().isEmpty()) {
-        return runClean(table, HoodieTimeline.getCleanRequestedInstant(instantTime), cleanerPlan);
-      }
-    }
-    return null;
-  }
-}
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java b/hudi-client/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java
deleted file mode 100644
index 0b27639..0000000
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java
+++ /dev/null
@@ -1,310 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hudi.table.action.commit;
-
-import org.apache.hudi.client.SparkTaskContextSupplier;
-import org.apache.hudi.client.WriteStatus;
-import org.apache.hudi.client.utils.SparkConfigUtils;
-import org.apache.hudi.common.model.HoodieCommitMetadata;
-import org.apache.hudi.common.model.HoodieKey;
-import org.apache.hudi.common.model.HoodieRecord;
-import org.apache.hudi.common.model.HoodieRecordPayload;
-import org.apache.hudi.common.model.HoodieWriteStat;
-import org.apache.hudi.common.model.WriteOperationType;
-import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
-import org.apache.hudi.common.table.timeline.HoodieInstant;
-import org.apache.hudi.common.table.timeline.HoodieInstant.State;
-import org.apache.hudi.common.util.CommitUtils;
-import org.apache.hudi.common.util.Option;
-import org.apache.hudi.config.HoodieWriteConfig;
-import org.apache.hudi.exception.HoodieCommitException;
-import org.apache.hudi.exception.HoodieIOException;
-import org.apache.hudi.exception.HoodieUpsertException;
-import org.apache.hudi.table.HoodieTable;
-import org.apache.hudi.table.WorkloadProfile;
-import org.apache.hudi.table.WorkloadStat;
-import org.apache.hudi.table.action.BaseActionExecutor;
-import org.apache.hudi.table.action.HoodieWriteMetadata;
-import org.apache.log4j.LogManager;
-import org.apache.log4j.Logger;
-import org.apache.spark.Partitioner;
-import org.apache.spark.api.java.JavaPairRDD;
-import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
-import org.apache.spark.storage.StorageLevel;
-import scala.Tuple2;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.nio.charset.StandardCharsets;
-import java.time.Duration;
-import java.time.Instant;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-
-public abstract class BaseCommitActionExecutor<T extends HoodieRecordPayload<T>, R>
-    extends BaseActionExecutor<R> {
-
-  private static final Logger LOG = LogManager.getLogger(BaseCommitActionExecutor.class);
-
-  protected final Option<Map<String, String>> extraMetadata;
-  private final WriteOperationType operationType;
-  protected final SparkTaskContextSupplier sparkTaskContextSupplier = new SparkTaskContextSupplier();
-
-  public BaseCommitActionExecutor(JavaSparkContext jsc, HoodieWriteConfig config,
-      HoodieTable table, String instantTime, WriteOperationType operationType,
-      Option<Map<String, String>> extraMetadata) {
-    super(jsc, config, table, instantTime);
-    this.operationType = operationType;
-    this.extraMetadata = extraMetadata;
-  }
-
-  public HoodieWriteMetadata execute(JavaRDD<HoodieRecord<T>> inputRecordsRDD) {
-    HoodieWriteMetadata result = new HoodieWriteMetadata();
-    // Cache the tagged records, so we don't end up computing both
-    // TODO: Consistent contract in HoodieWriteClient regarding preppedRecord storage level handling
-    if (inputRecordsRDD.getStorageLevel() == StorageLevel.NONE()) {
-      inputRecordsRDD.persist(StorageLevel.MEMORY_AND_DISK_SER());
-    } else {
-      LOG.info("RDD PreppedRecords was persisted at: " + inputRecordsRDD.getStorageLevel());
-    }
-
-    WorkloadProfile profile = null;
-    if (isWorkloadProfileNeeded()) {
-      profile = new WorkloadProfile(inputRecordsRDD);
-      LOG.info("Workload profile :" + profile);
-      saveWorkloadProfileMetadataToInflight(profile, instantTime);
-    }
-
-    // partition using the insert partitioner
-    final Partitioner partitioner = getPartitioner(profile);
-    JavaRDD<HoodieRecord<T>> partitionedRecords = partition(inputRecordsRDD, partitioner);
-    JavaRDD<WriteStatus> writeStatusRDD = partitionedRecords.mapPartitionsWithIndex((partition, recordItr) -> {
-      if (WriteOperationType.isChangingRecords(operationType)) {
-        return handleUpsertPartition(instantTime, partition, recordItr, partitioner);
-      } else {
-        return handleInsertPartition(instantTime, partition, recordItr, partitioner);
-      }
-    }, true).flatMap(List::iterator);
-
-    updateIndexAndCommitIfNeeded(writeStatusRDD, result);
-    return result;
-  }
-
-  /**
-   * Save the workload profile in an intermediate file (here re-using commit files) This is useful when performing
-   * rollback for MOR tables. Only updates are recorded in the workload profile metadata since updates to log blocks
-   * are unknown across batches Inserts (which are new parquet files) are rolled back based on commit time. // TODO :
-   * Create a new WorkloadProfile metadata file instead of using HoodieCommitMetadata
-   */
-  void saveWorkloadProfileMetadataToInflight(WorkloadProfile profile, String instantTime)
-      throws HoodieCommitException {
-    try {
-      HoodieCommitMetadata metadata = new HoodieCommitMetadata();
-      profile.getPartitionPaths().forEach(path -> {
-        WorkloadStat partitionStat = profile.getWorkloadStat(path.toString());
-        HoodieWriteStat insertStat = new HoodieWriteStat();
-        insertStat.setNumInserts(partitionStat.getNumInserts());
-        insertStat.setFileId("");
-        insertStat.setPrevCommit(HoodieWriteStat.NULL_COMMIT);
-        metadata.addWriteStat(path.toString(), insertStat);
-
-        partitionStat.getUpdateLocationToCount().forEach((key, value) -> {
-          HoodieWriteStat writeStat = new HoodieWriteStat();
-          writeStat.setFileId(key);
-          // TODO : Write baseCommitTime is possible here ?
-          writeStat.setPrevCommit(value.getKey());
-          writeStat.setNumUpdateWrites(value.getValue());
-          metadata.addWriteStat(path.toString(), writeStat);
-        });
-      });
-      metadata.setOperationType(operationType);
-
-      HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
-      String commitActionType = getCommitActionType();
-      HoodieInstant requested = new HoodieInstant(State.REQUESTED, commitActionType, instantTime);
-      activeTimeline.transitionRequestedToInflight(requested,
-          Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)),
-          config.shouldAllowMultiWriteOnSameInstant());
-    } catch (IOException io) {
-      throw new HoodieCommitException("Failed to commit " + instantTime + " unable to save inflight metadata ", io);
-    }
-  }
-
-  protected Partitioner getPartitioner(WorkloadProfile profile) {
-    if (WriteOperationType.isChangingRecords(operationType)) {
-      return getUpsertPartitioner(profile);
-    } else {
-      return getInsertPartitioner(profile);
-    }
-  }
-
-  private JavaRDD<HoodieRecord<T>> partition(JavaRDD<HoodieRecord<T>> dedupedRecords, Partitioner partitioner) {
-    JavaPairRDD<Tuple2, HoodieRecord<T>> mappedRDD = dedupedRecords.mapToPair(
-        record -> new Tuple2<>(new Tuple2<>(record.getKey(), Option.ofNullable(record.getCurrentLocation())), record));
-
-    JavaPairRDD<Tuple2, HoodieRecord<T>> partitionedRDD;
-    if (table.requireSortedRecords()) {
-      // Partition and sort within each partition as a single step. This is faster than partitioning first and then
-      // applying a sort.
-      Comparator<Tuple2> comparator = (Comparator<Tuple2> & Serializable)(t1, t2) -> {
-        HoodieKey key1 = (HoodieKey) t1._1;
-        HoodieKey key2 = (HoodieKey) t2._1;
-        return key1.getRecordKey().compareTo(key2.getRecordKey());
-      };
-
-      partitionedRDD = mappedRDD.repartitionAndSortWithinPartitions(partitioner, comparator);
-    } else {
-      // Partition only
-      partitionedRDD = mappedRDD.partitionBy(partitioner);
-    }
-
-    return partitionedRDD.map(Tuple2::_2);
-  }
-
-  protected void updateIndexAndCommitIfNeeded(JavaRDD<WriteStatus> writeStatusRDD, HoodieWriteMetadata result) {
-    // cache writeStatusRDD before updating index, so that all actions before this are not triggered again for future
-    // RDD actions that are performed after updating the index.
-    writeStatusRDD = writeStatusRDD.persist(SparkConfigUtils.getWriteStatusStorageLevel(config.getProps()));
-    Instant indexStartTime = Instant.now();
-    // Update the index back
-    JavaRDD<WriteStatus> statuses = ((HoodieTable<T>)table).getIndex().updateLocation(writeStatusRDD, jsc,
-        (HoodieTable<T>)table);
-    result.setIndexUpdateDuration(Duration.between(indexStartTime, Instant.now()));
-    result.setWriteStatuses(statuses);
-    result.setPartitionToReplaceFileIds(getPartitionToReplacedFileIds(statuses));
-    commitOnAutoCommit(result);
-  }
-
-  protected void commitOnAutoCommit(HoodieWriteMetadata result) {
-    if (config.shouldAutoCommit()) {
-      LOG.info("Auto commit enabled: Committing " + instantTime);
-      commit(extraMetadata, result);
-    } else {
-      LOG.info("Auto commit disabled for " + instantTime);
-    }
-  }
-
-  protected String getCommitActionType() {
-    return  table.getMetaClient().getCommitActionType();
-  }
-
-  protected void commit(Option<Map<String, String>> extraMetadata, HoodieWriteMetadata result) {
-    commit(extraMetadata, result, result.getWriteStatuses().map(WriteStatus::getStat).collect());
-  }
-
-  protected void commit(Option<Map<String, String>> extraMetadata, HoodieWriteMetadata result, List<HoodieWriteStat> writeStats) {
-    String actionType = getCommitActionType();
-    LOG.info("Committing " + instantTime + ", action Type " + actionType);
-    result.setCommitted(true);
-    result.setWriteStats(writeStats);
-    // Finalize write
-    finalizeWrite(instantTime, writeStats, result);
-
-    try {
-      LOG.info("Committing " + instantTime + ", action Type " + getCommitActionType());
-      HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
-      HoodieCommitMetadata metadata = CommitUtils.buildMetadata(writeStats, result.getPartitionToReplaceFileIds(),
-          extraMetadata, operationType, getSchemaToStoreInCommit(), getCommitActionType());
-
-      activeTimeline.saveAsComplete(new HoodieInstant(true, getCommitActionType(), instantTime),
-          Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
-      LOG.info("Committed " + instantTime);
-      result.setCommitMetadata(Option.of(metadata));
-    } catch (IOException e) {
-      throw new HoodieCommitException("Failed to complete commit " + config.getBasePath() + " at time " + instantTime,
-          e);
-    }
-  }
-
-  protected Map<String, List<String>> getPartitionToReplacedFileIds(JavaRDD<WriteStatus> writeStatuses) {
-    return Collections.emptyMap();
-  }
-
-  /**
-   * Finalize Write operation.
-   * @param instantTime Instant Time
-   * @param stats Hoodie Write Stat
-   */
-  protected void finalizeWrite(String instantTime, List<HoodieWriteStat> stats, HoodieWriteMetadata result) {
-    try {
-      Instant start = Instant.now();
-      table.finalizeWrite(jsc, instantTime, stats);
-      result.setFinalizeDuration(Duration.between(start, Instant.now()));
-    } catch (HoodieIOException ioe) {
-      throw new HoodieCommitException("Failed to complete commit " + instantTime + " due to finalize errors.", ioe);
-    }
-  }
-
-  /**
-   * By default, return the writer schema in Write Config for storing in commit.
-   */
-  protected String getSchemaToStoreInCommit() {
-    return config.getSchema();
-  }
-
-  protected boolean isWorkloadProfileNeeded() {
-    return true;
-  }
-
-  @SuppressWarnings("unchecked")
-  protected Iterator<List<WriteStatus>> handleUpsertPartition(String instantTime, Integer partition, Iterator recordItr,
-      Partitioner partitioner) {
-    UpsertPartitioner upsertPartitioner = (UpsertPartitioner) partitioner;
-    BucketInfo binfo = upsertPartitioner.getBucketInfo(partition);
-    BucketType btype = binfo.bucketType;
-    try {
-      if (btype.equals(BucketType.INSERT)) {
-        return handleInsert(binfo.fileIdPrefix, recordItr);
-      } else if (btype.equals(BucketType.UPDATE)) {
-        return handleUpdate(binfo.partitionPath, binfo.fileIdPrefix, recordItr);
-      } else {
-        throw new HoodieUpsertException("Unknown bucketType " + btype + " for partition :" + partition);
-      }
-    } catch (Throwable t) {
-      String msg = "Error upserting bucketType " + btype + " for partition :" + partition;
-      LOG.error(msg, t);
-      throw new HoodieUpsertException(msg, t);
-    }
-  }
-
-  protected Iterator<List<WriteStatus>> handleInsertPartition(String instantTime, Integer partition, Iterator recordItr,
-      Partitioner partitioner) {
-    return handleUpsertPartition(instantTime, partition, recordItr, partitioner);
-  }
-
-  /**
-   * Provides a partitioner to perform the upsert operation, based on the workload profile.
-   */
-  protected abstract Partitioner getUpsertPartitioner(WorkloadProfile profile);
-
-  /**
-   * Provides a partitioner to perform the insert operation, based on the workload profile.
-   */
-  protected abstract Partitioner getInsertPartitioner(WorkloadProfile profile);
-
-  protected abstract Iterator<List<WriteStatus>> handleInsert(String idPfx,
-      Iterator<HoodieRecord<T>> recordItr) throws Exception;
-
-  protected abstract Iterator<List<WriteStatus>> handleUpdate(String partitionPath, String fileId,
-      Iterator<HoodieRecord<T>> recordItr) throws IOException;
-}
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/BulkInsertHelper.java b/hudi-client/src/main/java/org/apache/hudi/table/action/commit/BulkInsertHelper.java
deleted file mode 100644
index 4683c82..0000000
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/BulkInsertHelper.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hudi.table.action.commit;
-
-import org.apache.hudi.client.WriteStatus;
-import org.apache.hudi.common.fs.FSUtils;
-import org.apache.hudi.common.model.HoodieRecord;
-import org.apache.hudi.common.model.HoodieRecordPayload;
-import org.apache.hudi.common.table.timeline.HoodieInstant;
-import org.apache.hudi.common.table.timeline.HoodieInstant.State;
-import org.apache.hudi.common.util.Option;
-import org.apache.hudi.config.HoodieWriteConfig;
-import org.apache.hudi.execution.bulkinsert.BulkInsertInternalPartitionerFactory;
-import org.apache.hudi.execution.bulkinsert.BulkInsertMapFunction;
-import org.apache.hudi.table.HoodieTable;
-import org.apache.hudi.table.BulkInsertPartitioner;
-import org.apache.hudi.table.action.HoodieWriteMetadata;
-import org.apache.spark.api.java.JavaRDD;
-
-import java.util.List;
-import java.util.stream.Collectors;
-import java.util.stream.IntStream;
-
-public class BulkInsertHelper<T extends HoodieRecordPayload<T>> {
-
-  public static <T extends HoodieRecordPayload<T>> HoodieWriteMetadata bulkInsert(JavaRDD<HoodieRecord<T>> inputRecords, String instantTime,
-                                                                                  HoodieTable<T> table, HoodieWriteConfig config,
-                                                                                  CommitActionExecutor<T> executor, boolean performDedupe,
-                                                                                  Option<BulkInsertPartitioner<T>> userDefinedBulkInsertPartitioner) {
-    HoodieWriteMetadata result = new HoodieWriteMetadata();
-
-    // De-dupe/merge if needed
-    JavaRDD<HoodieRecord<T>> dedupedRecords = inputRecords;
-
-    if (performDedupe) {
-      dedupedRecords = WriteHelper.combineOnCondition(config.shouldCombineBeforeInsert(), inputRecords,
-          config.getBulkInsertShuffleParallelism(), ((HoodieTable<T>)table));
-    }
-
-    final JavaRDD<HoodieRecord<T>> repartitionedRecords;
-    final int parallelism = config.getBulkInsertShuffleParallelism();
-    BulkInsertPartitioner partitioner = userDefinedBulkInsertPartitioner.isPresent()
-        ? userDefinedBulkInsertPartitioner.get()
-        : BulkInsertInternalPartitionerFactory.get(config.getBulkInsertSortMode());
-    repartitionedRecords = partitioner.repartitionRecords(dedupedRecords, parallelism);
-
-    // generate new file ID prefixes for each output partition
-    final List<String> fileIDPrefixes =
-        IntStream.range(0, parallelism).mapToObj(i -> FSUtils.createNewFileIdPfx()).collect(Collectors.toList());
-
-    table.getActiveTimeline().transitionRequestedToInflight(new HoodieInstant(State.REQUESTED,
-        table.getMetaClient().getCommitActionType(), instantTime), Option.empty(),
-        config.shouldAllowMultiWriteOnSameInstant());
-
-    JavaRDD<WriteStatus> writeStatusRDD = repartitionedRecords
-        .mapPartitionsWithIndex(new BulkInsertMapFunction<>(instantTime,
-            partitioner.arePartitionRecordsSorted(), config, table, fileIDPrefixes), true)
-        .flatMap(List::iterator);
-
-    executor.updateIndexAndCommitIfNeeded(writeStatusRDD, result);
-    return result;
-  }
-}
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/CommitActionExecutor.java b/hudi-client/src/main/java/org/apache/hudi/table/action/commit/CommitActionExecutor.java
deleted file mode 100644
index f35acaf..0000000
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/CommitActionExecutor.java
+++ /dev/null
@@ -1,139 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hudi.table.action.commit;
-
-import org.apache.hudi.client.WriteStatus;
-import org.apache.hudi.common.model.HoodieBaseFile;
-import org.apache.hudi.common.model.HoodieRecord;
-import org.apache.hudi.common.model.HoodieRecordPayload;
-import org.apache.hudi.common.model.WriteOperationType;
-import org.apache.hudi.common.util.Option;
-import org.apache.hudi.config.HoodieWriteConfig;
-import org.apache.hudi.exception.HoodieUpsertException;
-import org.apache.hudi.execution.LazyInsertIterable;
-import org.apache.hudi.io.HoodieMergeHandle;
-import org.apache.hudi.io.HoodieSortedMergeHandle;
-import org.apache.hudi.table.HoodieTable;
-import org.apache.hudi.table.WorkloadProfile;
-import org.apache.hudi.table.action.HoodieWriteMetadata;
-
-import org.apache.log4j.LogManager;
-import org.apache.log4j.Logger;
-import org.apache.spark.Partitioner;
-import org.apache.spark.api.java.JavaSparkContext;
-
-import java.io.IOException;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-
-public abstract class CommitActionExecutor<T extends HoodieRecordPayload<T>>
-    extends BaseCommitActionExecutor<T, HoodieWriteMetadata> {
-
-  private static final Logger LOG = LogManager.getLogger(CommitActionExecutor.class);
-
-  public CommitActionExecutor(JavaSparkContext jsc, HoodieWriteConfig config, HoodieTable table,
-                              String instantTime, WriteOperationType operationType) {
-    this(jsc, config, table, instantTime, operationType, Option.empty());
-  }
-
-  public CommitActionExecutor(JavaSparkContext jsc, HoodieWriteConfig config, HoodieTable table,
-                              String instantTime, WriteOperationType operationType,
-                              Option<Map<String, String>> extraMetadata) {
-    super(jsc, config, table, instantTime, operationType, extraMetadata);
-  }
-
-  @Override
-  public Iterator<List<WriteStatus>> handleUpdate(String partitionPath, String fileId,
-      Iterator<HoodieRecord<T>> recordItr)
-      throws IOException {
-    // This is needed since sometimes some buckets are never picked in getPartition() and end up with 0 records
-    if (!recordItr.hasNext()) {
-      LOG.info("Empty partition with fileId => " + fileId);
-      return Collections.singletonList((List<WriteStatus>) Collections.EMPTY_LIST).iterator();
-    }
-    // these are updates
-    HoodieMergeHandle upsertHandle = getUpdateHandle(partitionPath, fileId, recordItr);
-    return handleUpdateInternal(upsertHandle, fileId);
-  }
-
-  public Iterator<List<WriteStatus>> handleUpdate(String partitionPath, String fileId,
-      Map<String, HoodieRecord<T>> keyToNewRecords, HoodieBaseFile oldDataFile) throws IOException {
-    // these are updates
-    HoodieMergeHandle upsertHandle = getUpdateHandle(partitionPath, fileId, keyToNewRecords, oldDataFile);
-    return handleUpdateInternal(upsertHandle, fileId);
-  }
-
-  protected Iterator<List<WriteStatus>> handleUpdateInternal(HoodieMergeHandle upsertHandle, String fileId)
-      throws IOException {
-    if (upsertHandle.getOldFilePath() == null) {
-      throw new HoodieUpsertException(
-          "Error in finding the old file path at commit " + instantTime + " for fileId: " + fileId);
-    } else {
-      MergeHelper.runMerge(table, upsertHandle);
-    }
-
-    // TODO(vc): This needs to be revisited
-    if (upsertHandle.getWriteStatus().getPartitionPath() == null) {
-      LOG.info("Upsert Handle has partition path as null " + upsertHandle.getOldFilePath() + ", "
-          + upsertHandle.getWriteStatus());
-    }
-    return Collections.singletonList(Collections.singletonList(upsertHandle.getWriteStatus())).iterator();
-  }
-
-  protected HoodieMergeHandle getUpdateHandle(String partitionPath, String fileId, Iterator<HoodieRecord<T>> recordItr) {
-    if (table.requireSortedRecords()) {
-      return new HoodieSortedMergeHandle<>(config, instantTime, (HoodieTable<T>)table, recordItr, partitionPath, fileId, sparkTaskContextSupplier);
-    } else {
-      return new HoodieMergeHandle<>(config, instantTime, (HoodieTable<T>)table, recordItr, partitionPath, fileId, sparkTaskContextSupplier);
-    }
-  }
-
-  protected HoodieMergeHandle getUpdateHandle(String partitionPath, String fileId,
-      Map<String, HoodieRecord<T>> keyToNewRecords, HoodieBaseFile dataFileToBeMerged) {
-    return new HoodieMergeHandle<>(config, instantTime, (HoodieTable<T>)table, keyToNewRecords,
-        partitionPath, fileId, dataFileToBeMerged, sparkTaskContextSupplier);
-  }
-
-  @Override
-  public Iterator<List<WriteStatus>> handleInsert(String idPfx, Iterator<HoodieRecord<T>> recordItr)
-      throws Exception {
-    // This is needed since sometimes some buckets are never picked in getPartition() and end up with 0 records
-    if (!recordItr.hasNext()) {
-      LOG.info("Empty partition");
-      return Collections.singletonList((List<WriteStatus>) Collections.EMPTY_LIST).iterator();
-    }
-    return new LazyInsertIterable<>(recordItr, config, instantTime, (HoodieTable<T>)table, idPfx,
-        sparkTaskContextSupplier);
-  }
-
-  @Override
-  public Partitioner getUpsertPartitioner(WorkloadProfile profile) {
-    if (profile == null) {
-      throw new HoodieUpsertException("Need workload profile to construct the upsert partitioner.");
-    }
-    return new UpsertPartitioner(profile, jsc, table, config);
-  }
-
-  @Override
-  public Partitioner getInsertPartitioner(WorkloadProfile profile) {
-    return getUpsertPartitioner(profile);
-  }
-}
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPreppedCommitActionExecutor.java b/hudi-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPreppedCommitActionExecutor.java
deleted file mode 100644
index d8470ea..0000000
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPreppedCommitActionExecutor.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hudi.table.action.commit;
-
-import org.apache.hudi.common.model.HoodieRecord;
-import org.apache.hudi.common.model.HoodieRecordPayload;
-import org.apache.hudi.common.model.WriteOperationType;
-import org.apache.hudi.config.HoodieWriteConfig;
-import org.apache.hudi.table.HoodieTable;
-
-import org.apache.hudi.table.action.HoodieWriteMetadata;
-import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
-
-public class UpsertPreppedCommitActionExecutor<T extends HoodieRecordPayload<T>>
-    extends CommitActionExecutor<T> {
-
-  private final JavaRDD<HoodieRecord<T>> preppedRecords;
-
-  public UpsertPreppedCommitActionExecutor(JavaSparkContext jsc,
-      HoodieWriteConfig config, HoodieTable table,
-      String instantTime, JavaRDD<HoodieRecord<T>> preppedRecords) {
-    super(jsc, config, table, instantTime, WriteOperationType.UPSERT_PREPPED);
-    this.preppedRecords = preppedRecords;
-  }
-
-  public HoodieWriteMetadata execute() {
-    return super.execute(preppedRecords);
-  }
-}
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/WriteHelper.java b/hudi-client/src/main/java/org/apache/hudi/table/action/commit/WriteHelper.java
deleted file mode 100644
index 92dcbb6..0000000
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/WriteHelper.java
+++ /dev/null
@@ -1,106 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hudi.table.action.commit;
-
-import org.apache.hudi.common.model.HoodieKey;
-import org.apache.hudi.common.model.HoodieRecord;
-import org.apache.hudi.common.model.HoodieRecordPayload;
-import org.apache.hudi.exception.HoodieUpsertException;
-import org.apache.hudi.index.HoodieIndex;
-import org.apache.hudi.table.HoodieTable;
-
-import org.apache.hudi.table.action.HoodieWriteMetadata;
-import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
-
-import java.time.Duration;
-import java.time.Instant;
-import scala.Tuple2;
-
-public class WriteHelper<T extends HoodieRecordPayload<T>> {
-
-  public static <T extends HoodieRecordPayload<T>> HoodieWriteMetadata write(String instantTime,
-                                                                             JavaRDD<HoodieRecord<T>> inputRecordsRDD, JavaSparkContext jsc,
-                                                                             HoodieTable<T> table, boolean shouldCombine,
-                                                                             int shuffleParallelism, CommitActionExecutor<T> executor, boolean performTagging) {
-    try {
-      // De-dupe/merge if needed
-      JavaRDD<HoodieRecord<T>> dedupedRecords =
-          combineOnCondition(shouldCombine, inputRecordsRDD, shuffleParallelism, table);
-
-      Instant lookupBegin = Instant.now();
-      JavaRDD<HoodieRecord<T>> taggedRecords = dedupedRecords;
-      if (performTagging) {
-        // perform index loop up to get existing location of records
-        taggedRecords = tag(dedupedRecords, jsc, table);
-      }
-      Duration indexLookupDuration = Duration.between(lookupBegin, Instant.now());
-
-      HoodieWriteMetadata result = executor.execute(taggedRecords);
-      result.setIndexLookupDuration(indexLookupDuration);
-      return result;
-    } catch (Throwable e) {
-      if (e instanceof HoodieUpsertException) {
-        throw (HoodieUpsertException) e;
-      }
-      throw new HoodieUpsertException("Failed to upsert for commit time " + instantTime, e);
-    }
-  }
-
-  private static <T extends HoodieRecordPayload<T>> JavaRDD<HoodieRecord<T>> tag(
-      JavaRDD<HoodieRecord<T>> dedupedRecords, JavaSparkContext jsc, HoodieTable<T> table) {
-    // perform index loop up to get existing location of records
-    return table.getIndex().tagLocation(dedupedRecords, jsc, table);
-  }
-
-  public static <T extends HoodieRecordPayload<T>> JavaRDD<HoodieRecord<T>> combineOnCondition(
-      boolean condition, JavaRDD<HoodieRecord<T>> records, int parallelism, HoodieTable<T> table) {
-    return condition ? deduplicateRecords(records, table, parallelism) : records;
-  }
-
-  /**
-   * Deduplicate Hoodie records, using the given deduplication function.
-   *
-   * @param records hoodieRecords to deduplicate
-   * @param parallelism parallelism or partitions to be used while reducing/deduplicating
-   * @return RDD of HoodieRecord already be deduplicated
-   */
-  public static <T extends HoodieRecordPayload<T>> JavaRDD<HoodieRecord<T>> deduplicateRecords(
-      JavaRDD<HoodieRecord<T>> records, HoodieTable<T> table, int parallelism) {
-    return deduplicateRecords(records, table.getIndex(), parallelism);
-  }
-
-  public static <T extends HoodieRecordPayload<T>> JavaRDD<HoodieRecord<T>> deduplicateRecords(
-      JavaRDD<HoodieRecord<T>> records, HoodieIndex<T> index, int parallelism) {
-    boolean isIndexingGlobal = index.isGlobal();
-    return records.mapToPair(record -> {
-      HoodieKey hoodieKey = record.getKey();
-      // If index used is global, then records are expected to differ in their partitionPath
-      Object key = isIndexingGlobal ? hoodieKey.getRecordKey() : hoodieKey;
-      return new Tuple2<>(key, record);
-    }).reduceByKey((rec1, rec2) -> {
-      @SuppressWarnings("unchecked")
-      T reducedData = (T) rec1.getData().preCombine(rec2.getData());
-      // we cannot allow the user to change the key or partitionPath, since that will affect
-      // everything
-      // so pick it from one of the records.
-      return new HoodieRecord<T>(rec1.getKey(), reducedData);
-    }, parallelism).map(Tuple2::_2);
-  }
-}
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionActionExecutor.java b/hudi-client/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionActionExecutor.java
deleted file mode 100644
index 0d7f6be..0000000
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/compact/ScheduleCompactionActionExecutor.java
+++ /dev/null
@@ -1,122 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hudi.table.action.compact;
-
-import org.apache.hudi.avro.model.HoodieCompactionPlan;
-import org.apache.hudi.common.table.timeline.HoodieInstant;
-import org.apache.hudi.common.table.timeline.HoodieTimeline;
-import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
-import org.apache.hudi.common.table.view.SyncableFileSystemView;
-import org.apache.hudi.common.util.Option;
-import org.apache.hudi.common.util.ValidationUtils;
-import org.apache.hudi.config.HoodieWriteConfig;
-import org.apache.hudi.exception.HoodieCompactionException;
-import org.apache.hudi.exception.HoodieIOException;
-import org.apache.hudi.table.HoodieTable;
-import org.apache.hudi.table.action.BaseActionExecutor;
-import org.apache.log4j.LogManager;
-import org.apache.log4j.Logger;
-import org.apache.spark.api.java.JavaSparkContext;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-import java.util.stream.Collectors;
-
-public class ScheduleCompactionActionExecutor extends BaseActionExecutor<Option<HoodieCompactionPlan>> {
-
-  private static final Logger LOG = LogManager.getLogger(ScheduleCompactionActionExecutor.class);
-
-  private final Option<Map<String, String>> extraMetadata;
-
-  public ScheduleCompactionActionExecutor(JavaSparkContext jsc,
-                                          HoodieWriteConfig config,
-                                          HoodieTable<?> table,
-                                          String instantTime,
-                                          Option<Map<String, String>> extraMetadata) {
-    super(jsc, config, table, instantTime);
-    this.extraMetadata = extraMetadata;
-  }
-
-  private HoodieCompactionPlan scheduleCompaction() {
-    LOG.info("Checking if compaction needs to be run on " + config.getBasePath());
-    Option<HoodieInstant> lastCompaction = table.getActiveTimeline().getCommitTimeline()
-        .filterCompletedInstants().lastInstant();
-    String lastCompactionTs = "0";
-    if (lastCompaction.isPresent()) {
-      lastCompactionTs = lastCompaction.get().getTimestamp();
-    }
-
-    int deltaCommitsSinceLastCompaction = table.getActiveTimeline().getDeltaCommitTimeline()
-        .findInstantsAfter(lastCompactionTs, Integer.MAX_VALUE).countInstants();
-    if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction) {
-      LOG.info("Not scheduling compaction as only " + deltaCommitsSinceLastCompaction
-          + " delta commits was found since last compaction " + lastCompactionTs + ". Waiting for "
-          + config.getInlineCompactDeltaCommitMax());
-      return new HoodieCompactionPlan();
-    }
-
-    LOG.info("Generating compaction plan for merge on read table " + config.getBasePath());
-    HoodieMergeOnReadTableCompactor compactor = new HoodieMergeOnReadTableCompactor();
-    try {
-      return compactor.generateCompactionPlan(jsc, table, config, instantTime,
-          ((SyncableFileSystemView) table.getSliceView()).getPendingCompactionOperations()
-              .map(instantTimeOpPair -> instantTimeOpPair.getValue().getFileGroupId())
-              .collect(Collectors.toSet()));
-
-    } catch (IOException e) {
-      throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e);
-    }
-  }
-
-  @Override
-  public Option<HoodieCompactionPlan> execute() {
-    // if there are inflight writes, their instantTime must not be less than that of compaction instant time
-    table.getActiveTimeline().getCommitsTimeline().filterPendingExcludingCompaction().firstInstant()
-        .ifPresent(earliestInflight -> ValidationUtils.checkArgument(
-            HoodieTimeline.compareTimestamps(earliestInflight.getTimestamp(), HoodieTimeline.GREATER_THAN, instantTime),
-            "Earliest write inflight instant time must be later than compaction time. Earliest :" + earliestInflight
-                + ", Compaction scheduled at " + instantTime));
-
-    // Committed and pending compaction instants should have strictly lower timestamps
-    List<HoodieInstant> conflictingInstants = table.getActiveTimeline()
-        .getCommitsAndCompactionTimeline().getInstants()
-        .filter(instant -> HoodieTimeline.compareTimestamps(
-            instant.getTimestamp(), HoodieTimeline.GREATER_THAN_OR_EQUALS, instantTime))
-        .collect(Collectors.toList());
-    ValidationUtils.checkArgument(conflictingInstants.isEmpty(),
-        "Following instants have timestamps >= compactionInstant (" + instantTime + ") Instants :"
-            + conflictingInstants);
-
-    HoodieCompactionPlan plan = scheduleCompaction();
-    if (plan != null && (plan.getOperations() != null) && (!plan.getOperations().isEmpty())) {
-      extraMetadata.ifPresent(plan::setExtraMetadata);
-      HoodieInstant compactionInstant =
-          new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, instantTime);
-      try {
-        table.getActiveTimeline().saveToCompactionRequested(compactionInstant,
-            TimelineMetadataUtils.serializeCompactionPlan(plan));
-      } catch (IOException ioe) {
-        throw new HoodieIOException("Exception scheduling compaction", ioe);
-      }
-      return Option.of(plan);
-    }
-    return Option.empty();
-  }
-}
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/BulkInsertDeltaCommitActionExecutor.java b/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/BulkInsertDeltaCommitActionExecutor.java
deleted file mode 100644
index 01ff1fa..0000000
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/BulkInsertDeltaCommitActionExecutor.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hudi.table.action.deltacommit;
-
-import java.util.Map;
-import org.apache.hudi.common.model.HoodieRecord;
-import org.apache.hudi.common.model.HoodieRecordPayload;
-import org.apache.hudi.common.model.WriteOperationType;
-import org.apache.hudi.common.util.Option;
-import org.apache.hudi.config.HoodieWriteConfig;
-import org.apache.hudi.exception.HoodieInsertException;
-import org.apache.hudi.table.HoodieTable;
-import org.apache.hudi.table.BulkInsertPartitioner;
-
-import org.apache.hudi.table.action.commit.BulkInsertHelper;
-import org.apache.hudi.table.action.HoodieWriteMetadata;
-import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
-
-public class BulkInsertDeltaCommitActionExecutor<T extends HoodieRecordPayload<T>>
-    extends DeltaCommitActionExecutor<T> {
-
-  private final JavaRDD<HoodieRecord<T>> inputRecordsRDD;
-  private final Option<BulkInsertPartitioner<T>> bulkInsertPartitioner;
-
-  public BulkInsertDeltaCommitActionExecutor(JavaSparkContext jsc, HoodieWriteConfig config, HoodieTable table,
-                                             String instantTime, JavaRDD<HoodieRecord<T>> inputRecordsRDD,
-                                             Option<BulkInsertPartitioner<T>> bulkInsertPartitioner)  {
-    this(jsc, config, table, instantTime, inputRecordsRDD, bulkInsertPartitioner, Option.empty());
-  }
-
-  public BulkInsertDeltaCommitActionExecutor(JavaSparkContext jsc, HoodieWriteConfig config, HoodieTable table,
-                                             String instantTime, JavaRDD<HoodieRecord<T>> inputRecordsRDD,
-                                             Option<BulkInsertPartitioner<T>> bulkInsertPartitioner,
-                                             Option<Map<String, String>> extraMetadata) {
-    super(jsc, config, table, instantTime, WriteOperationType.BULK_INSERT, extraMetadata);
-    this.inputRecordsRDD = inputRecordsRDD;
-    this.bulkInsertPartitioner = bulkInsertPartitioner;
-  }
-
-  @Override
-  public HoodieWriteMetadata execute() {
-    try {
-      return BulkInsertHelper.bulkInsert(inputRecordsRDD, instantTime, (HoodieTable<T>) table, config,
-          this, true, bulkInsertPartitioner);
-    } catch (HoodieInsertException ie) {
-      throw ie;
-    } catch (Throwable e) {
-      throw new HoodieInsertException("Failed to bulk insert for commit time " + instantTime, e);
-    }
-  }
-}
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/UpsertPreppedDeltaCommitActionExecutor.java b/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/UpsertPreppedDeltaCommitActionExecutor.java
deleted file mode 100644
index d1773f9..0000000
--- a/hudi-client/src/main/java/org/apache/hudi/table/action/deltacommit/UpsertPreppedDeltaCommitActionExecutor.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hudi.table.action.deltacommit;
-
-import org.apache.hudi.common.model.HoodieRecord;
-import org.apache.hudi.common.model.HoodieRecordPayload;
-import org.apache.hudi.common.model.WriteOperationType;
-import org.apache.hudi.config.HoodieWriteConfig;
-import org.apache.hudi.table.HoodieTable;
-import org.apache.hudi.table.action.HoodieWriteMetadata;
-import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
-
-public class UpsertPreppedDeltaCommitActionExecutor<T extends HoodieRecordPayload<T>>
-    extends DeltaCommitActionExecutor<T> {
-
-  private final JavaRDD<HoodieRecord<T>> preppedRecords;
-
-  public UpsertPreppedDeltaCommitActionExecutor(JavaSparkContext jsc,
-      HoodieWriteConfig config, HoodieTable table,
-      String instantTime, JavaRDD<HoodieRecord<T>> preppedRecords) {
-    super(jsc, config, table, instantTime, WriteOperationType.UPSERT_PREPPED);
-    this.preppedRecords = preppedRecords;
-  }
-
-  public HoodieWriteMetadata execute() {
-    return super.execute(preppedRecords);
-  }
-}
diff --git a/hudi-examples/pom.xml b/hudi-examples/pom.xml
index 5fe3ddc..ba13290 100644
--- a/hudi-examples/pom.xml
+++ b/hudi-examples/pom.xml
@@ -129,7 +129,13 @@
 
     <dependency>
       <groupId>org.apache.hudi</groupId>
-      <artifactId>hudi-client</artifactId>
+      <artifactId>hudi-client-common</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hudi</groupId>
+      <artifactId>hudi-spark-client</artifactId>
       <version>${project.version}</version>
     </dependency>
 
diff --git a/hudi-examples/src/main/java/org/apache/hudi/examples/spark/HoodieWriteClientExample.java b/hudi-examples/src/main/java/org/apache/hudi/examples/spark/HoodieWriteClientExample.java
index 655e549..b606c52 100644
--- a/hudi-examples/src/main/java/org/apache/hudi/examples/spark/HoodieWriteClientExample.java
+++ b/hudi-examples/src/main/java/org/apache/hudi/examples/spark/HoodieWriteClientExample.java
@@ -18,8 +18,9 @@
 
 package org.apache.hudi.examples.spark;
 
-import org.apache.hudi.client.HoodieWriteClient;
+import org.apache.hudi.client.SparkRDDWriteClient;
 import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.HoodieAvroPayload;
 import org.apache.hudi.common.model.HoodieKey;
@@ -48,7 +49,7 @@
 
 
 /**
- * Simple examples of #{@link HoodieWriteClient}.
+ * Simple examples of #{@link SparkRDDWriteClient}.
  *
  * To run this example, you should
  * <pre>
@@ -94,7 +95,7 @@
               .withDeleteParallelism(2).forTable(tableName)
               .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
               .withCompactionConfig(HoodieCompactionConfig.newBuilder().archiveCommitsWith(20, 30).build()).build();
-      HoodieWriteClient<HoodieAvroPayload> client = new HoodieWriteClient<>(jsc, cfg);
+      SparkRDDWriteClient<HoodieAvroPayload> client = new SparkRDDWriteClient<>(new HoodieSparkEngineContext(jsc), cfg);
 
       // inserts
       String newCommitTime = client.startCommit();
diff --git a/hudi-integ-test/pom.xml b/hudi-integ-test/pom.xml
index fff52a4..e9fcc61 100644
--- a/hudi-integ-test/pom.xml
+++ b/hudi-integ-test/pom.xml
@@ -191,7 +191,13 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hudi</groupId>
-      <artifactId>hudi-client</artifactId>
+      <artifactId>hudi-client-common</artifactId>
+      <version>${project.version}</version>
+      <type>test-jar</type>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hudi</groupId>
+      <artifactId>hudi-spark-client</artifactId>
       <version>${project.version}</version>
       <type>test-jar</type>
     </dependency>
diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteWriter.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteWriter.java
index b22faca..ff4f0a6 100644
--- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteWriter.java
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteWriter.java
@@ -27,8 +27,9 @@
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hudi.avro.model.HoodieCompactionPlan;
 import org.apache.hudi.client.HoodieReadClient;
-import org.apache.hudi.client.HoodieWriteClient;
+import org.apache.hudi.client.SparkRDDWriteClient;
 import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
 import org.apache.hudi.common.util.Option;
@@ -50,12 +51,12 @@
 /**
  * A writer abstraction for the Hudi test suite. This class wraps different implementations of writers used to perform
  * write operations into the target hudi dataset. Current supported writers are {@link HoodieDeltaStreamerWrapper}
- * and {@link HoodieWriteClient}.
+ * and {@link SparkRDDWriteClient}.
  */
 public class HoodieTestSuiteWriter {
 
   private HoodieDeltaStreamerWrapper deltaStreamerWrapper;
-  private HoodieWriteClient writeClient;
+  private SparkRDDWriteClient writeClient;
   protected HoodieTestSuiteConfig cfg;
   private Option<String> lastCheckpoint;
   private HoodieReadClient hoodieReadClient;
@@ -76,10 +77,11 @@
     // We ensure that only 1 instance of HoodieWriteClient is instantiated for a HoodieTestSuiteWriter
     // This does not instantiate a HoodieWriteClient until a
     // {@link HoodieDeltaStreamer#commit(HoodieWriteClient, JavaRDD, Option)} is invoked.
+    HoodieSparkEngineContext context = new HoodieSparkEngineContext(jsc);
     this.deltaStreamerWrapper = new HoodieDeltaStreamerWrapper(cfg, jsc);
-    this.hoodieReadClient = new HoodieReadClient(jsc, cfg.targetBasePath);
+    this.hoodieReadClient = new HoodieReadClient(context, cfg.targetBasePath);
     if (!cfg.useDeltaStreamer) {
-      this.writeClient = new HoodieWriteClient(jsc, getHoodieClientConfig(cfg, props, schema), rollbackInflight);
+      this.writeClient = new SparkRDDWriteClient(context, getHoodieClientConfig(cfg, props, schema), rollbackInflight);
     }
     this.cfg = cfg;
     this.configuration = jsc.hadoopConfiguration();
@@ -162,7 +164,7 @@
         }
       }
       if (instantTime.isPresent()) {
-        return writeClient.compact(instantTime.get());
+        return (JavaRDD<WriteStatus>) writeClient.compact(instantTime.get());
       } else {
         return null;
       }
@@ -183,19 +185,19 @@
     if (!cfg.useDeltaStreamer) {
       Map<String, String> extraMetadata = new HashMap<>();
       /** Store the checkpoint in the commit metadata just like
-       * {@link HoodieDeltaStreamer#commit(HoodieWriteClient, JavaRDD, Option)} **/
+       * {@link HoodieDeltaStreamer#commit(SparkRDDWriteClient, JavaRDD, Option)} **/
       extraMetadata.put(HoodieDeltaStreamerWrapper.CHECKPOINT_KEY, lastCheckpoint.get());
       writeClient.commit(instantTime.get(), records, Option.of(extraMetadata));
     }
   }
 
-  public HoodieWriteClient getWriteClient(DagNode dagNode) throws IllegalAccessException {
+  public SparkRDDWriteClient getWriteClient(DagNode dagNode) throws IllegalAccessException {
     if (cfg.useDeltaStreamer & !allowWriteClientAccess(dagNode)) {
       throw new IllegalAccessException("cannot access write client when testing in deltastreamer mode");
     }
     synchronized (this) {
       if (writeClient == null) {
-        this.writeClient = new HoodieWriteClient(this.sparkContext, getHoodieClientConfig(cfg, props, schema), false);
+        this.writeClient = new SparkRDDWriteClient(new HoodieSparkEngineContext(this.sparkContext), getHoodieClientConfig(cfg, props, schema), false);
       }
     }
     return writeClient;
diff --git a/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/reader/TestDFSHoodieDatasetInputReader.java b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/reader/TestDFSHoodieDatasetInputReader.java
index 15e6f70..1caf8f8 100644
--- a/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/reader/TestDFSHoodieDatasetInputReader.java
+++ b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/reader/TestDFSHoodieDatasetInputReader.java
@@ -26,8 +26,9 @@
 import org.apache.avro.Schema;
 import org.apache.avro.generic.GenericRecord;
 import org.apache.hudi.avro.HoodieAvroUtils;
-import org.apache.hudi.client.HoodieWriteClient;
+import org.apache.hudi.client.SparkRDDWriteClient;
 import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
 import org.apache.hudi.common.testutils.HoodieTestUtils;
@@ -70,7 +71,7 @@
   public void testSimpleHoodieDatasetReader() throws Exception {
 
     HoodieWriteConfig config = makeHoodieClientConfig();
-    HoodieWriteClient client = new HoodieWriteClient(jsc, config);
+    SparkRDDWriteClient client = new SparkRDDWriteClient(new HoodieSparkEngineContext(jsc), config);
     String commitTime = client.startCommit();
     HoodieTestDataGenerator generator = new HoodieTestDataGenerator();
     // Insert 100 records across 3 partitions
diff --git a/hudi-spark/pom.xml b/hudi-spark/pom.xml
index a3c8464..0942327 100644
--- a/hudi-spark/pom.xml
+++ b/hudi-spark/pom.xml
@@ -168,7 +168,12 @@
     <!-- Hoodie -->
     <dependency>
       <groupId>org.apache.hudi</groupId>
-      <artifactId>hudi-client</artifactId>
+      <artifactId>hudi-client-common</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hudi</groupId>
+      <artifactId>hudi-spark-client</artifactId>
       <version>${project.version}</version>
     </dependency>
     <dependency>
@@ -340,7 +345,15 @@
     <!-- Hoodie - Test -->
     <dependency>
       <groupId>org.apache.hudi</groupId>
-      <artifactId>hudi-client</artifactId>
+      <artifactId>hudi-client-common</artifactId>
+      <version>${project.version}</version>
+      <classifier>tests</classifier>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hudi</groupId>
+      <artifactId>hudi-spark-client</artifactId>
       <version>${project.version}</version>
       <classifier>tests</classifier>
       <type>test-jar</type>
diff --git a/hudi-spark/src/main/java/org/apache/hudi/DataSourceUtils.java b/hudi-spark/src/main/java/org/apache/hudi/DataSourceUtils.java
index b61d3a6..a4d4715 100644
--- a/hudi-spark/src/main/java/org/apache/hudi/DataSourceUtils.java
+++ b/hudi-spark/src/main/java/org/apache/hudi/DataSourceUtils.java
@@ -22,8 +22,9 @@
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hudi.client.HoodieReadClient;
-import org.apache.hudi.client.HoodieWriteClient;
 import org.apache.hudi.client.HoodieWriteResult;
+import org.apache.hudi.client.SparkRDDWriteClient;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.config.TypedProperties;
 import org.apache.hudi.common.model.HoodieKey;
 import org.apache.hudi.common.model.HoodieRecord;
@@ -171,9 +172,9 @@
         .withProps(parameters).build();
   }
 
-  public static HoodieWriteClient createHoodieClient(JavaSparkContext jssc, String schemaStr, String basePath,
-      String tblName, Map<String, String> parameters) {
-    return new HoodieWriteClient<>(jssc, createHoodieConfig(schemaStr, basePath, tblName, parameters), true);
+  public static SparkRDDWriteClient createHoodieClient(JavaSparkContext jssc, String schemaStr, String basePath,
+                                                       String tblName, Map<String, String> parameters) {
+    return new SparkRDDWriteClient<>(new HoodieSparkEngineContext(jssc), createHoodieConfig(schemaStr, basePath, tblName, parameters), true);
   }
 
   public static String getCommitActionType(WriteOperationType operation, HoodieTableType tableType) {
@@ -184,7 +185,7 @@
     }
   }
 
-  public static HoodieWriteResult doWriteOperation(HoodieWriteClient client, JavaRDD<HoodieRecord> hoodieRecords,
+  public static HoodieWriteResult doWriteOperation(SparkRDDWriteClient client, JavaRDD<HoodieRecord> hoodieRecords,
                                                    String instantTime, WriteOperationType operation) throws HoodieException {
     switch (operation) {
       case BULK_INSERT:
@@ -202,7 +203,7 @@
     }
   }
 
-  public static HoodieWriteResult doDeleteOperation(HoodieWriteClient client, JavaRDD<HoodieKey> hoodieKeys,
+  public static HoodieWriteResult doDeleteOperation(SparkRDDWriteClient client, JavaRDD<HoodieKey> hoodieKeys,
       String instantTime) {
     return new HoodieWriteResult(client.delete(hoodieKeys, instantTime));
   }
@@ -224,7 +225,7 @@
   public static JavaRDD<HoodieRecord> dropDuplicates(JavaSparkContext jssc, JavaRDD<HoodieRecord> incomingHoodieRecords,
       HoodieWriteConfig writeConfig) {
     try {
-      HoodieReadClient client = new HoodieReadClient<>(jssc, writeConfig);
+      HoodieReadClient client = new HoodieReadClient<>(new HoodieSparkEngineContext(jssc), writeConfig);
       return client.tagLocation(incomingHoodieRecords)
           .filter(r -> !((HoodieRecord<HoodieRecordPayload>) r).isCurrentLocationKnown());
     } catch (TableNotFoundException e) {
diff --git a/hudi-spark/src/main/java/org/apache/hudi/async/SparkStreamingAsyncCompactService.java b/hudi-spark/src/main/java/org/apache/hudi/async/SparkStreamingAsyncCompactService.java
index ae0ad73..d1a415b 100644
--- a/hudi-spark/src/main/java/org/apache/hudi/async/SparkStreamingAsyncCompactService.java
+++ b/hudi-spark/src/main/java/org/apache/hudi/async/SparkStreamingAsyncCompactService.java
@@ -18,8 +18,10 @@
 
 package org.apache.hudi.async;
 
-import org.apache.hudi.client.HoodieWriteClient;
-import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.hudi.client.AbstractCompactor;
+import org.apache.hudi.client.AbstractHoodieWriteClient;
+import org.apache.hudi.client.HoodieSparkCompactor;
+import org.apache.hudi.client.common.HoodieEngineContext;
 
 /**
  * Async Compaction Service used by Structured Streaming. Here, async compaction is run in daemon mode to prevent
@@ -29,7 +31,12 @@
 
   private static final long serialVersionUID = 1L;
 
-  public SparkStreamingAsyncCompactService(JavaSparkContext jssc, HoodieWriteClient client) {
-    super(jssc, client, true);
+  public SparkStreamingAsyncCompactService(HoodieEngineContext context, AbstractHoodieWriteClient client) {
+    super(context, client, true);
+  }
+
+  @Override
+  protected AbstractCompactor createCompactor(AbstractHoodieWriteClient client) {
+    return new HoodieSparkCompactor(client);
   }
 }
diff --git a/hudi-spark/src/main/java/org/apache/hudi/bootstrap/SparkParquetBootstrapDataProvider.java b/hudi-spark/src/main/java/org/apache/hudi/bootstrap/SparkParquetBootstrapDataProvider.java
index 3a5a796..022abe3 100644
--- a/hudi-spark/src/main/java/org/apache/hudi/bootstrap/SparkParquetBootstrapDataProvider.java
+++ b/hudi-spark/src/main/java/org/apache/hudi/bootstrap/SparkParquetBootstrapDataProvider.java
@@ -23,6 +23,7 @@
 import org.apache.hudi.avro.HoodieAvroUtils;
 import org.apache.hudi.avro.model.HoodieFileStatus;
 import org.apache.hudi.client.bootstrap.FullRecordBootstrapDataProvider;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.bootstrap.FileStatusUtils;
 import org.apache.hudi.common.config.TypedProperties;
 import org.apache.hudi.common.model.HoodieRecord;
@@ -32,7 +33,6 @@
 
 import org.apache.avro.generic.GenericRecord;
 import org.apache.spark.api.java.JavaRDD;
-import org.apache.spark.api.java.JavaSparkContext;
 import org.apache.spark.rdd.RDD;
 import org.apache.spark.sql.Dataset;
 import org.apache.spark.sql.SparkSession;
@@ -43,18 +43,18 @@
 /**
  * Spark Data frame based bootstrap input provider.
  */
-public class SparkParquetBootstrapDataProvider extends FullRecordBootstrapDataProvider {
+public class SparkParquetBootstrapDataProvider extends FullRecordBootstrapDataProvider<JavaRDD<HoodieRecord>> {
 
   private final transient SparkSession sparkSession;
 
   public SparkParquetBootstrapDataProvider(TypedProperties props,
-                                           JavaSparkContext jsc) {
-    super(props, jsc);
-    this.sparkSession = SparkSession.builder().config(jsc.getConf()).getOrCreate();
+                                           HoodieSparkEngineContext context) {
+    super(props, context);
+    this.sparkSession = SparkSession.builder().config(context.getJavaSparkContext().getConf()).getOrCreate();
   }
 
   @Override
-  public JavaRDD<HoodieRecord> generateInputRecordRDD(String tableName, String sourceBasePath,
+  public JavaRDD<HoodieRecord> generateInputRecords(String tableName, String sourceBasePath,
       List<Pair<String, List<HoodieFileStatus>>> partitionPathsWithFiles) {
     String[] filePaths = partitionPathsWithFiles.stream().map(Pair::getValue)
         .flatMap(f -> f.stream().map(fs -> FileStatusUtils.toPath(fs.getPath()).toString()))
diff --git a/hudi-spark/src/main/java/org/apache/hudi/internal/HoodieDataSourceInternalWriter.java b/hudi-spark/src/main/java/org/apache/hudi/internal/HoodieDataSourceInternalWriter.java
index a4bfd59..e8cbff8 100644
--- a/hudi-spark/src/main/java/org/apache/hudi/internal/HoodieDataSourceInternalWriter.java
+++ b/hudi-spark/src/main/java/org/apache/hudi/internal/HoodieDataSourceInternalWriter.java
@@ -20,7 +20,8 @@
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hudi.DataSourceUtils;
-import org.apache.hudi.client.HoodieWriteClient;
+import org.apache.hudi.client.SparkRDDWriteClient;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.model.HoodieWriteStat;
 import org.apache.hudi.common.model.WriteOperationType;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
@@ -30,6 +31,7 @@
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.table.HoodieSparkTable;
 import org.apache.hudi.table.HoodieTable;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
@@ -59,7 +61,7 @@
   private final HoodieTableMetaClient metaClient;
   private final HoodieWriteConfig writeConfig;
   private final StructType structType;
-  private final HoodieWriteClient writeClient;
+  private final SparkRDDWriteClient writeClient;
   private final HoodieTable hoodieTable;
   private final WriteOperationType operationType;
 
@@ -69,11 +71,11 @@
     this.writeConfig = writeConfig;
     this.structType = structType;
     this.operationType = WriteOperationType.BULK_INSERT;
-    this.writeClient  = new HoodieWriteClient<>(new JavaSparkContext(sparkSession.sparkContext()), writeConfig, true);
+    this.writeClient  = new SparkRDDWriteClient<>(new HoodieSparkEngineContext(new JavaSparkContext(sparkSession.sparkContext())), writeConfig, true);
     writeClient.setOperationType(operationType);
     writeClient.startCommitWithTime(instantTime);
     this.metaClient = new HoodieTableMetaClient(configuration, writeConfig.getBasePath());
-    this.hoodieTable = HoodieTable.create(metaClient, writeConfig, metaClient.getHadoopConf());
+    this.hoodieTable = HoodieSparkTable.create(writeConfig, new HoodieSparkEngineContext(new JavaSparkContext(sparkSession.sparkContext())), metaClient);
   }
 
   @Override
diff --git a/hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java b/hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java
index 26efa5e..7c67f9a 100644
--- a/hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java
+++ b/hudi-spark/src/main/java/org/apache/hudi/keygen/KeyGenerator.java
@@ -36,7 +36,7 @@
  * Abstract class to extend for plugging in extraction of {@link HoodieKey} from an Avro record.
  */
 @PublicAPIClass(maturity = ApiMaturityLevel.STABLE)
-public abstract class KeyGenerator implements Serializable, KeyGeneratorInterface {
+public abstract class KeyGenerator implements Serializable, SparkKeyGeneratorInterface {
 
   private static final String STRUCT_NAME = "hoodieRowTopLevelField";
   private static final String NAMESPACE = "hoodieRow";
diff --git a/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala b/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala
index df7960e..7173a28 100644
--- a/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala
+++ b/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala
@@ -27,7 +27,8 @@
 import org.apache.hadoop.hive.conf.HiveConf
 import org.apache.hudi.DataSourceWriteOptions._
 import org.apache.hudi.avro.HoodieAvroUtils
-import org.apache.hudi.client.{HoodieWriteClient, HoodieWriteResult}
+import org.apache.hudi.client.{SparkRDDWriteClient, HoodieWriteResult}
+import org.apache.hudi.client.{SparkRDDWriteClient, WriteStatus}
 import org.apache.hudi.common.config.TypedProperties
 import org.apache.hudi.common.model.{HoodieRecordPayload, HoodieTableType, WriteOperationType}
 import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
@@ -59,11 +60,11 @@
             parameters: Map[String, String],
             df: DataFrame,
             hoodieTableConfigOpt: Option[HoodieTableConfig] = Option.empty,
-            hoodieWriteClient: Option[HoodieWriteClient[HoodieRecordPayload[Nothing]]] = Option.empty,
-            asyncCompactionTriggerFn: Option[Function1[HoodieWriteClient[HoodieRecordPayload[Nothing]], Unit]] = Option.empty
+            hoodieWriteClient: Option[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]] = Option.empty,
+            asyncCompactionTriggerFn: Option[Function1[SparkRDDWriteClient[HoodieRecordPayload[Nothing]], Unit]] = Option.empty
            )
   : (Boolean, common.util.Option[String], common.util.Option[String],
-     HoodieWriteClient[HoodieRecordPayload[Nothing]], HoodieTableConfig) = {
+    SparkRDDWriteClient[HoodieRecordPayload[Nothing]], HoodieTableConfig) = {
 
     val sparkContext = sqlContext.sparkContext
     val path = parameters.get("path")
@@ -126,7 +127,7 @@
       }
       // scalastyle:on
 
-      val (writeResult, writeClient: HoodieWriteClient[HoodieRecordPayload[Nothing]]) =
+      val (writeResult, writeClient: SparkRDDWriteClient[HoodieRecordPayload[Nothing]]) =
         if (operation != WriteOperationType.DELETE) {
           // register classes & schemas
           val (structName, nameSpace) = AvroConversionUtils.getAvroRecordNameAndNamespace(tblName)
@@ -151,7 +152,7 @@
           // Create a HoodieWriteClient & issue the write.
           val client = hoodieWriteClient.getOrElse(DataSourceUtils.createHoodieClient(jsc, schema.toString, path.get,
             tblName, mapAsJavaMap(parameters)
-          )).asInstanceOf[HoodieWriteClient[HoodieRecordPayload[Nothing]]]
+          )).asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]]
 
           if (isAsyncCompactionEnabled(client, tableConfig, parameters, jsc.hadoopConfiguration())) {
             asyncCompactionTriggerFn.get.apply(client)
@@ -190,7 +191,7 @@
           // Create a HoodieWriteClient & issue the delete.
           val client = hoodieWriteClient.getOrElse(DataSourceUtils.createHoodieClient(jsc,
             Schema.create(Schema.Type.NULL).toString, path.get, tblName,
-            mapAsJavaMap(parameters))).asInstanceOf[HoodieWriteClient[HoodieRecordPayload[Nothing]]]
+            mapAsJavaMap(parameters))).asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]]
 
           if (isAsyncCompactionEnabled(client, tableConfig, parameters, jsc.hadoopConfiguration())) {
             asyncCompactionTriggerFn.get.apply(client)
@@ -389,7 +390,7 @@
 
   private def commitAndPerformPostOperations(writeResult: HoodieWriteResult,
                                              parameters: Map[String, String],
-                                             client: HoodieWriteClient[HoodieRecordPayload[Nothing]],
+                                             client: SparkRDDWriteClient[HoodieRecordPayload[Nothing]],
                                              tableConfig: HoodieTableConfig,
                                              jsc: JavaSparkContext,
                                              tableInstantInfo: TableInstantInfo
@@ -446,7 +447,7 @@
     }
   }
 
-  private def isAsyncCompactionEnabled(client: HoodieWriteClient[HoodieRecordPayload[Nothing]],
+  private def isAsyncCompactionEnabled(client: SparkRDDWriteClient[HoodieRecordPayload[Nothing]],
                                        tableConfig: HoodieTableConfig,
                                        parameters: Map[String, String], configuration: Configuration) : Boolean = {
     log.info(s"Config.isInlineCompaction ? ${client.getConfig.isInlineCompaction}")
diff --git a/hudi-spark/src/main/scala/org/apache/hudi/HoodieStreamingSink.scala b/hudi-spark/src/main/scala/org/apache/hudi/HoodieStreamingSink.scala
index 1600ab0..9f57fb5 100644
--- a/hudi-spark/src/main/scala/org/apache/hudi/HoodieStreamingSink.scala
+++ b/hudi-spark/src/main/scala/org/apache/hudi/HoodieStreamingSink.scala
@@ -20,7 +20,8 @@
 import java.util.function.{Function, Supplier}
 
 import org.apache.hudi.async.{AsyncCompactService, SparkStreamingAsyncCompactService}
-import org.apache.hudi.client.HoodieWriteClient
+import org.apache.hudi.client.SparkRDDWriteClient
+import org.apache.hudi.client.common.HoodieSparkEngineContext
 import org.apache.hudi.common.model.HoodieRecordPayload
 import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
 import org.apache.hudi.common.table.timeline.HoodieInstant.State
@@ -60,7 +61,7 @@
     }
 
   private var asyncCompactorService : AsyncCompactService = _
-  private var writeClient : Option[HoodieWriteClient[HoodieRecordPayload[Nothing]]] = Option.empty
+  private var writeClient : Option[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]] = Option.empty
   private var hoodieTableConfig : Option[HoodieTableConfig] = Option.empty
 
   override def addBatch(batchId: Long, data: DataFrame): Unit = this.synchronized {
@@ -153,10 +154,10 @@
     }
   }
 
-  protected def triggerAsyncCompactor(client: HoodieWriteClient[HoodieRecordPayload[Nothing]]): Unit = {
+  protected def triggerAsyncCompactor(client: SparkRDDWriteClient[HoodieRecordPayload[Nothing]]): Unit = {
     if (null == asyncCompactorService) {
       log.info("Triggering Async compaction !!")
-      asyncCompactorService = new SparkStreamingAsyncCompactService(new JavaSparkContext(sqlContext.sparkContext),
+      asyncCompactorService = new SparkStreamingAsyncCompactService(new HoodieSparkEngineContext(new JavaSparkContext(sqlContext.sparkContext)),
         client)
       asyncCompactorService.start(new Function[java.lang.Boolean, java.lang.Boolean] {
         override def apply(errored: lang.Boolean): lang.Boolean = {
diff --git a/hudi-spark/src/main/scala/org/apache/hudi/IncrementalRelation.scala b/hudi-spark/src/main/scala/org/apache/hudi/IncrementalRelation.scala
index e894b06..e113d4a 100644
--- a/hudi-spark/src/main/scala/org/apache/hudi/IncrementalRelation.scala
+++ b/hudi-spark/src/main/scala/org/apache/hudi/IncrementalRelation.scala
@@ -17,7 +17,6 @@
 
 package org.apache.hudi
 
-
 import com.google.common.collect.Lists
 import org.apache.avro.Schema
 import org.apache.hadoop.fs.GlobPattern
@@ -30,10 +29,11 @@
 import org.apache.hudi.common.util.ParquetUtils
 import org.apache.hudi.config.HoodieWriteConfig
 import org.apache.hudi.exception.HoodieException
-import org.apache.hudi.table.HoodieTable
-
 import org.apache.hadoop.fs.GlobPattern
+import org.apache.hudi.client.common.HoodieSparkEngineContext
+import org.apache.hudi.table.HoodieSparkTable
 import org.apache.log4j.LogManager
+import org.apache.spark.api.java.JavaSparkContext
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.sources.{BaseRelation, TableScan}
 import org.apache.spark.sql.types.{StringType, StructField, StructType}
@@ -64,8 +64,9 @@
     throw new HoodieException("Incremental view not implemented yet, for merge-on-read tables")
   }
   // TODO : Figure out a valid HoodieWriteConfig
-  private val hoodieTable = HoodieTable.create(metaClient, HoodieWriteConfig.newBuilder().withPath(basePath).build(),
-    sqlContext.sparkContext.hadoopConfiguration)
+  private val hoodieTable = HoodieSparkTable.create(HoodieWriteConfig.newBuilder().withPath(basePath).build(),
+    new HoodieSparkEngineContext(new JavaSparkContext(sqlContext.sparkContext)),
+    metaClient)
   private val commitTimeline = hoodieTable.getMetaClient.getCommitTimeline.filterCompletedInstants()
   if (commitTimeline.empty()) {
     throw new HoodieException("No instants to incrementally pull")
diff --git a/hudi-spark/src/test/java/org/apache/hudi/TestDataSourceUtils.java b/hudi-spark/src/test/java/org/apache/hudi/TestDataSourceUtils.java
index 9ff114e..97948b9 100644
--- a/hudi-spark/src/test/java/org/apache/hudi/TestDataSourceUtils.java
+++ b/hudi-spark/src/test/java/org/apache/hudi/TestDataSourceUtils.java
@@ -19,7 +19,7 @@
 package org.apache.hudi;
 
 import org.apache.hudi.avro.HoodieAvroUtils;
-import org.apache.hudi.client.HoodieWriteClient;
+import org.apache.hudi.client.SparkRDDWriteClient;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.model.WriteOperationType;
@@ -63,7 +63,7 @@
 public class TestDataSourceUtils {
 
   @Mock
-  private HoodieWriteClient hoodieWriteClient;
+  private SparkRDDWriteClient hoodieWriteClient;
 
   @Mock
   private JavaRDD<HoodieRecord> hoodieRecords;
@@ -172,7 +172,7 @@
   }
 
   public static class NoOpBulkInsertPartitioner<T extends HoodieRecordPayload>
-      implements BulkInsertPartitioner<T> {
+      implements BulkInsertPartitioner<JavaRDD<HoodieRecord<T>>> {
 
     @Override
     public JavaRDD<HoodieRecord<T>> repartitionRecords(JavaRDD<HoodieRecord<T>> records, int outputSparkPartitions) {
diff --git a/hudi-spark/src/test/java/org/apache/hudi/client/TestBootstrap.java b/hudi-spark/src/test/java/org/apache/hudi/client/TestBootstrap.java
index 14f36d4..7e13a5e 100644
--- a/hudi-spark/src/test/java/org/apache/hudi/client/TestBootstrap.java
+++ b/hudi-spark/src/test/java/org/apache/hudi/client/TestBootstrap.java
@@ -25,6 +25,7 @@
 import org.apache.hudi.client.bootstrap.selector.BootstrapModeSelector;
 import org.apache.hudi.client.bootstrap.selector.FullRecordBootstrapModeSelector;
 import org.apache.hudi.client.bootstrap.selector.MetadataOnlyBootstrapModeSelector;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.bootstrap.FileStatusUtils;
 import org.apache.hudi.common.bootstrap.index.BootstrapIndex;
 import org.apache.hudi.common.config.TypedProperties;
@@ -166,7 +167,7 @@
       df.write().format("parquet").mode(SaveMode.Overwrite).save(srcPath);
     }
     String filePath = FileStatusUtils.toPath(BootstrapUtils.getAllLeafFoldersWithFiles(metaClient, metaClient.getFs(),
-            srcPath, jsc).stream().findAny().map(p -> p.getValue().stream().findAny())
+            srcPath, context).stream().findAny().map(p -> p.getValue().stream().findAny())
             .orElse(null).get().getPath()).toString();
     ParquetFileReader reader = ParquetFileReader.open(metaClient.getHadoopConf(), new Path(filePath));
     MessageType schema = reader.getFooter().getFileMetaData().getSchema();
@@ -249,7 +250,7 @@
             .withBootstrapParallelism(3)
             .withBootstrapModeSelector(bootstrapModeSelectorClass).build())
         .build();
-    HoodieWriteClient client = new HoodieWriteClient(jsc, config);
+    SparkRDDWriteClient client = new SparkRDDWriteClient(context, config);
     client.bootstrap(Option.empty());
     checkBootstrapResults(totalRecords, schema, bootstrapCommitInstantTs, checkNumRawFiles, numInstantsAfterBootstrap,
         numInstantsAfterBootstrap, timestamp, timestamp, deltaCommit, bootstrapInstants);
@@ -260,14 +261,14 @@
     client.rollBackInflightBootstrap();
     metaClient.reloadActiveTimeline();
     assertEquals(0, metaClient.getCommitsTimeline().countInstants());
-    assertEquals(0L, BootstrapUtils.getAllLeafFoldersWithFiles(metaClient, metaClient.getFs(), basePath, jsc)
+    assertEquals(0L, BootstrapUtils.getAllLeafFoldersWithFiles(metaClient, metaClient.getFs(), basePath, context)
             .stream().flatMap(f -> f.getValue().stream()).count());
 
     BootstrapIndex index = BootstrapIndex.getBootstrapIndex(metaClient);
     assertFalse(index.useIndex());
 
     // Run bootstrap again
-    client = new HoodieWriteClient(jsc, config);
+    client = new SparkRDDWriteClient(context, config);
     client.bootstrap(Option.empty());
 
     metaClient.reloadActiveTimeline();
@@ -286,7 +287,7 @@
     String updateSPath = tmpFolder.toAbsolutePath().toString() + "/data2";
     generateNewDataSetAndReturnSchema(updateTimestamp, totalRecords, partitions, updateSPath);
     JavaRDD<HoodieRecord> updateBatch =
-        generateInputBatch(jsc, BootstrapUtils.getAllLeafFoldersWithFiles(metaClient, metaClient.getFs(), updateSPath, jsc),
+        generateInputBatch(jsc, BootstrapUtils.getAllLeafFoldersWithFiles(metaClient, metaClient.getFs(), updateSPath, context),
                 schema);
     String newInstantTs = client.startCommit();
     client.upsert(updateBatch, newInstantTs);
@@ -348,7 +349,7 @@
     original.registerTempTable("original");
     if (checkNumRawFiles) {
       List<HoodieFileStatus> files = BootstrapUtils.getAllLeafFoldersWithFiles(metaClient, metaClient.getFs(),
-          bootstrapBasePath, jsc).stream().flatMap(x -> x.getValue().stream()).collect(Collectors.toList());
+          bootstrapBasePath, context).stream().flatMap(x -> x.getValue().stream()).collect(Collectors.toList());
       assertEquals(files.size() * numVersions,
           sqlContext.sql("select distinct _hoodie_file_name from bootstrapped").count());
     }
@@ -466,18 +467,19 @@
     assertEquals(totalRecords, seenKeys.size());
   }
 
-  public static class TestFullBootstrapDataProvider extends FullRecordBootstrapDataProvider {
+  public static class TestFullBootstrapDataProvider extends FullRecordBootstrapDataProvider<JavaRDD<HoodieRecord>> {
 
-    public TestFullBootstrapDataProvider(TypedProperties props, JavaSparkContext jsc) {
-      super(props, jsc);
+    public TestFullBootstrapDataProvider(TypedProperties props, HoodieSparkEngineContext context) {
+      super(props, context);
     }
 
     @Override
-    public JavaRDD<HoodieRecord> generateInputRecordRDD(String tableName, String sourceBasePath,
+    public JavaRDD<HoodieRecord> generateInputRecords(String tableName, String sourceBasePath,
         List<Pair<String, List<HoodieFileStatus>>> partitionPaths) {
       String filePath = FileStatusUtils.toPath(partitionPaths.stream().flatMap(p -> p.getValue().stream())
           .findAny().get().getPath()).toString();
       ParquetFileReader reader = null;
+      JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context);
       try {
         reader = ParquetFileReader.open(jsc.hadoopConfiguration(), new Path(filePath));
       } catch (IOException e) {
diff --git a/hudi-spark/src/test/java/org/apache/hudi/internal/TestHoodieBulkInsertDataInternalWriter.java b/hudi-spark/src/test/java/org/apache/hudi/internal/TestHoodieBulkInsertDataInternalWriter.java
index 884e11c..5a5d8b2 100644
--- a/hudi-spark/src/test/java/org/apache/hudi/internal/TestHoodieBulkInsertDataInternalWriter.java
+++ b/hudi-spark/src/test/java/org/apache/hudi/internal/TestHoodieBulkInsertDataInternalWriter.java
@@ -23,6 +23,7 @@
 import org.apache.hudi.common.model.HoodieWriteStat;
 import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
 import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.table.HoodieSparkTable;
 import org.apache.hudi.table.HoodieTable;
 import org.apache.hudi.testutils.HoodieClientTestHarness;
 
@@ -76,7 +77,7 @@
   public void testDataInternalWriter() throws IOException {
     // init config and table
     HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
-    HoodieTable table = HoodieTable.create(metaClient, cfg, hadoopConf);
+    HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
     // execute N rounds
     for (int i = 0; i < 5; i++) {
       String instantTime = "00" + i;
@@ -121,7 +122,7 @@
   public void testGlobalFailure() throws IOException {
     // init config and table
     HoodieWriteConfig cfg = getConfigBuilder(basePath).build();
-    HoodieTable table = HoodieTable.create(metaClient, cfg, hadoopConf);
+    HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
     String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[0];
 
     String instantTime = "001";
diff --git a/hudi-spark/src/test/scala/org/apache/hudi/functional/HoodieSparkSqlWriterSuite.scala b/hudi-spark/src/test/scala/org/apache/hudi/functional/HoodieSparkSqlWriterSuite.scala
index bcd83db..15872dd 100644
--- a/hudi-spark/src/test/scala/org/apache/hudi/functional/HoodieSparkSqlWriterSuite.scala
+++ b/hudi-spark/src/test/scala/org/apache/hudi/functional/HoodieSparkSqlWriterSuite.scala
@@ -22,7 +22,7 @@
 
 import org.apache.commons.io.FileUtils
 import org.apache.hudi.DataSourceWriteOptions._
-import org.apache.hudi.client.HoodieWriteClient
+import org.apache.hudi.client.SparkRDDWriteClient
 import org.apache.hudi.common.model.{HoodieRecord, HoodieRecordPayload}
 import org.apache.hudi.common.testutils.HoodieTestDataGenerator
 import org.apache.hudi.config.HoodieWriteConfig
@@ -250,7 +250,7 @@
             schema.toString,
             path.toAbsolutePath.toString,
             hoodieFooTableName,
-            mapAsJavaMap(fooTableParams)).asInstanceOf[HoodieWriteClient[HoodieRecordPayload[Nothing]]])
+            mapAsJavaMap(fooTableParams)).asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]])
 
           // write to Hudi
           HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, df, Option.empty,
diff --git a/hudi-utilities/pom.xml b/hudi-utilities/pom.xml
index 02100a2..ab51475 100644
--- a/hudi-utilities/pom.xml
+++ b/hudi-utilities/pom.xml
@@ -94,7 +94,12 @@
     <!-- Hoodie -->
     <dependency>
       <groupId>org.apache.hudi</groupId>
-      <artifactId>hudi-client</artifactId>
+      <artifactId>hudi-client-common</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hudi</groupId>
+      <artifactId>hudi-spark-client</artifactId>
       <version>${project.version}</version>
     </dependency>
     <dependency>
@@ -347,7 +352,15 @@
     <!-- Hoodie - Test -->
     <dependency>
       <groupId>org.apache.hudi</groupId>
-      <artifactId>hudi-client</artifactId>
+      <artifactId>hudi-client-common</artifactId>
+      <version>${project.version}</version>
+      <classifier>tests</classifier>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hudi</groupId>
+      <artifactId>hudi-spark-client</artifactId>
       <version>${project.version}</version>
       <classifier>tests</classifier>
       <type>test-jar</type>
diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HDFSParquetImporter.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HDFSParquetImporter.java
index c15585b..af66c17 100644
--- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HDFSParquetImporter.java
+++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HDFSParquetImporter.java
@@ -18,8 +18,10 @@
 
 package org.apache.hudi.utilities;
 
-import org.apache.hudi.client.HoodieWriteClient;
+import org.apache.hudi.client.SparkRDDWriteClient;
 import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.HoodieJsonPayload;
 import org.apache.hudi.common.config.TypedProperties;
 import org.apache.hudi.common.fs.FSUtils;
@@ -142,7 +144,7 @@
       // Get schema.
       String schemaStr = UtilHelpers.parseSchema(fs, cfg.schemaFile);
 
-      HoodieWriteClient client =
+      SparkRDDWriteClient client =
           UtilHelpers.createHoodieClient(jsc, cfg.targetPath, schemaStr, cfg.parallelism, Option.empty(), props);
 
       JavaRDD<HoodieRecord<HoodieRecordPayload>> hoodieRecords = buildHoodieRecordsForImport(jsc, schemaStr);
@@ -166,7 +168,8 @@
     AvroReadSupport.setAvroReadSchema(jsc.hadoopConfiguration(), (new Schema.Parser().parse(schemaStr)));
     ParquetInputFormat.setReadSupportClass(job, (AvroReadSupport.class));
 
-    jsc.setJobGroup(this.getClass().getSimpleName(), "Build records for import");
+    HoodieEngineContext context = new HoodieSparkEngineContext(jsc);
+    context.setJobStatus(this.getClass().getSimpleName(), "Build records for import");
     return jsc.newAPIHadoopFile(cfg.srcPath, ParquetInputFormat.class, Void.class, GenericRecord.class,
             job.getConfiguration())
         // To reduce large number of tasks.
@@ -203,8 +206,8 @@
    * @param hoodieRecords Hoodie Records
    * @param <T> Type
    */
-  protected <T extends HoodieRecordPayload> JavaRDD<WriteStatus> load(HoodieWriteClient client, String instantTime,
-      JavaRDD<HoodieRecord<T>> hoodieRecords) {
+  protected <T extends HoodieRecordPayload> JavaRDD<WriteStatus> load(SparkRDDWriteClient client, String instantTime,
+                                                                      JavaRDD<HoodieRecord<T>> hoodieRecords) {
     switch (cfg.command.toLowerCase()) {
       case "upsert": {
         return client.upsert(hoodieRecords, instantTime);
diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCleaner.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCleaner.java
index bf1fedb..94798ea 100644
--- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCleaner.java
+++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCleaner.java
@@ -18,7 +18,8 @@
 
 package org.apache.hudi.utilities;
 
-import org.apache.hudi.client.HoodieWriteClient;
+import org.apache.hudi.client.SparkRDDWriteClient;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.config.TypedProperties;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.config.HoodieWriteConfig;
@@ -68,7 +69,7 @@
 
   public void run() {
     HoodieWriteConfig hoodieCfg = getHoodieClientConfig();
-    HoodieWriteClient client = new HoodieWriteClient<>(jssc, hoodieCfg, false);
+    SparkRDDWriteClient client = new SparkRDDWriteClient<>(new HoodieSparkEngineContext(jssc), hoodieCfg, false);
     client.clean();
   }
 
diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCompactionAdminTool.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCompactionAdminTool.java
index d30355f..d3e4dba 100644
--- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCompactionAdminTool.java
+++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCompactionAdminTool.java
@@ -21,6 +21,7 @@
 import org.apache.hudi.client.CompactionAdminClient;
 import org.apache.hudi.client.CompactionAdminClient.RenameOpResult;
 import org.apache.hudi.client.CompactionAdminClient.ValidationOpResult;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.HoodieFileGroupId;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
@@ -60,7 +61,7 @@
    */
   public void run(JavaSparkContext jsc) throws Exception {
     HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.basePath);
-    try (CompactionAdminClient admin = new CompactionAdminClient(jsc, cfg.basePath)) {
+    try (CompactionAdminClient admin = new CompactionAdminClient(new HoodieSparkEngineContext(jsc), cfg.basePath)) {
       final FileSystem fs = FSUtils.getFs(cfg.basePath, jsc.hadoopConfiguration());
       if (cfg.outputPath != null && fs.exists(new Path(cfg.outputPath))) {
         throw new IllegalStateException("Output File Path already exists");
diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCompactor.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCompactor.java
index a091124..30d5445 100644
--- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCompactor.java
+++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCompactor.java
@@ -18,7 +18,7 @@
 
 package org.apache.hudi.utilities;
 
-import org.apache.hudi.client.HoodieWriteClient;
+import org.apache.hudi.client.SparkRDDWriteClient;
 import org.apache.hudi.client.WriteStatus;
 import org.apache.hudi.common.config.TypedProperties;
 import org.apache.hudi.common.fs.FSUtils;
@@ -130,15 +130,15 @@
   private int doCompact(JavaSparkContext jsc) throws Exception {
     // Get schema.
     String schemaStr = UtilHelpers.parseSchema(fs, cfg.schemaFile);
-    HoodieWriteClient client =
+    SparkRDDWriteClient client =
         UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism, Option.empty(), props);
-    JavaRDD<WriteStatus> writeResponse = client.compact(cfg.compactionInstantTime);
+    JavaRDD<WriteStatus> writeResponse = (JavaRDD<WriteStatus>) client.compact(cfg.compactionInstantTime);
     return UtilHelpers.handleErrors(jsc, cfg.compactionInstantTime, writeResponse);
   }
 
   private int doSchedule(JavaSparkContext jsc) throws Exception {
     // Get schema.
-    HoodieWriteClient client =
+    SparkRDDWriteClient client =
         UtilHelpers.createHoodieClient(jsc, cfg.basePath, "", cfg.parallelism, Option.of(cfg.strategyClassName), props);
     client.scheduleCompactionAtInstant(cfg.compactionInstantTime, Option.empty());
     return 0;
diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotCopier.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotCopier.java
index 916d019..05b4627 100644
--- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotCopier.java
+++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotCopier.java
@@ -18,6 +18,8 @@
 
 package org.apache.hudi.utilities;
 
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.config.SerializableConfiguration;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.HoodieBaseFile;
@@ -97,8 +99,10 @@
         fs.delete(new Path(outputDir), true);
       }
 
-      jsc.setJobGroup(this.getClass().getSimpleName(), "Creating a snapshot");
-      jsc.parallelize(partitions, partitions.size()).flatMap(partition -> {
+      HoodieEngineContext context = new HoodieSparkEngineContext(jsc);
+      context.setJobStatus(this.getClass().getSimpleName(), "Creating a snapshot");
+
+      List<Tuple2<String, String>> filesToCopy = context.flatMap(partitions, partition -> {
         // Only take latest version files <= latestCommit.
         FileSystem fs1 = FSUtils.getFs(baseDir, serConf.newCopy());
         List<Tuple2<String, String>> filePaths = new ArrayList<>();
@@ -112,8 +116,10 @@
           filePaths.add(new Tuple2<>(partition, partitionMetaFile.toString()));
         }
 
-        return filePaths.iterator();
-      }).foreach(tuple -> {
+        return filePaths.stream();
+      }, partitions.size());
+
+      context.foreach(filesToCopy, tuple -> {
         String partition = tuple._1();
         Path sourceFilePath = new Path(tuple._2());
         Path toPartitionPath = new Path(outputDir, partition);
@@ -124,8 +130,8 @@
         }
         FileUtil.copy(ifs, sourceFilePath, ifs, new Path(toPartitionPath, sourceFilePath.getName()), false,
             ifs.getConf());
-      });
-
+      }, filesToCopy.size());
+      
       // Also copy the .commit files
       LOG.info(String.format("Copying .commit files which are no-late-than %s.", latestCommitTimestamp));
       FileStatus[] commitFilesToCopy =
diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotExporter.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotExporter.java
index 0743839..cf69dd2 100644
--- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotExporter.java
+++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotExporter.java
@@ -18,6 +18,8 @@
 
 package org.apache.hudi.utilities;
 
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.config.SerializableConfiguration;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.HoodieBaseFile;
@@ -175,7 +177,8 @@
         ? defaultPartitioner
         : ReflectionUtils.loadClass(cfg.outputPartitioner);
 
-    jsc.setJobGroup(this.getClass().getSimpleName(), "Exporting as non-HUDI dataset");
+    HoodieEngineContext context = new HoodieSparkEngineContext(jsc);
+    context.setJobStatus(this.getClass().getSimpleName(), "Exporting as non-HUDI dataset");
     final BaseFileOnlyView fsView = getBaseFileOnlyView(jsc, cfg);
     Iterator<String> exportingFilePaths = jsc
         .parallelize(partitions, partitions.size())
@@ -193,14 +196,16 @@
 
   private void exportAsHudi(JavaSparkContext jsc, Config cfg, List<String> partitions, String latestCommitTimestamp) throws IOException {
     final BaseFileOnlyView fsView = getBaseFileOnlyView(jsc, cfg);
-    final SerializableConfiguration serConf = new SerializableConfiguration(jsc.hadoopConfiguration());
-    jsc.setJobGroup(this.getClass().getSimpleName(), "Exporting as HUDI dataset");
-    jsc.parallelize(partitions, partitions.size()).flatMap(partition -> {
+
+    final HoodieEngineContext context = new HoodieSparkEngineContext(jsc);
+    final SerializableConfiguration serConf = context.getHadoopConf();
+    context.setJobStatus(this.getClass().getSimpleName(), "Exporting as HUDI dataset");
+
+    List<Tuple2<String, String>> files = context.flatMap(partitions, partition -> {
       // Only take latest version files <= latestCommit.
       List<Tuple2<String, String>> filePaths = new ArrayList<>();
       Stream<HoodieBaseFile> dataFiles = fsView.getLatestBaseFilesBeforeOrOn(partition, latestCommitTimestamp);
       dataFiles.forEach(hoodieDataFile -> filePaths.add(new Tuple2<>(partition, hoodieDataFile.getPath())));
-
       // also need to copy over partition metadata
       Path partitionMetaFile =
           new Path(new Path(cfg.sourceBasePath, partition), HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE);
@@ -208,9 +213,10 @@
       if (fs.exists(partitionMetaFile)) {
         filePaths.add(new Tuple2<>(partition, partitionMetaFile.toString()));
       }
+      return filePaths.stream();
+    }, partitions.size());
 
-      return filePaths.iterator();
-    }).foreach(tuple -> {
+    context.foreach(files, tuple -> {
       String partition = tuple._1();
       Path sourceFilePath = new Path(tuple._2());
       Path toPartitionPath = new Path(cfg.targetOutputPath, partition);
@@ -221,7 +227,7 @@
       }
       FileUtil.copy(fs, sourceFilePath, fs, new Path(toPartitionPath, sourceFilePath.getName()), false,
           fs.getConf());
-    });
+    }, files.size());
 
     // Also copy the .commit files
     LOG.info(String.format("Copying .commit files which are no-late-than %s.", latestCommitTimestamp));
diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieWithTimelineServer.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieWithTimelineServer.java
index 06c1084..8974061 100644
--- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieWithTimelineServer.java
+++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieWithTimelineServer.java
@@ -18,6 +18,8 @@
 
 package org.apache.hudi.utilities;
 
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.util.ValidationUtils;
 
 import com.beust.jcommander.JCommander;
@@ -86,8 +88,10 @@
     System.out.println("Driver Hostname is :" + driverHost);
     List<String> messages = new ArrayList<>();
     IntStream.range(0, cfg.numPartitions).forEach(i -> messages.add("Hello World"));
-    jsc.setJobGroup(this.getClass().getSimpleName(), "Sending requests to driver host");
-    List<String> gotMessages = jsc.parallelize(messages).map(msg -> sendRequest(driverHost, cfg.serverPort)).collect();
+
+    HoodieEngineContext context = new HoodieSparkEngineContext(jsc);
+    context.setJobStatus(this.getClass().getSimpleName(), "Sending requests to driver host");
+    List<String> gotMessages = context.map(messages, msg -> sendRequest(driverHost, cfg.serverPort), messages.size());
     System.out.println("Got Messages :" + gotMessages);
     ValidationUtils.checkArgument(gotMessages.equals(messages), "Got expected reply from Server");
   }
diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/UtilHelpers.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/UtilHelpers.java
index 0f4a64c..29fc195 100644
--- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/UtilHelpers.java
+++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/UtilHelpers.java
@@ -20,8 +20,9 @@
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hudi.AvroConversionUtils;
-import org.apache.hudi.client.HoodieWriteClient;
+import org.apache.hudi.client.SparkRDDWriteClient;
 import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.config.DFSPropertiesConfiguration;
 import org.apache.hudi.common.config.TypedProperties;
 import org.apache.hudi.common.util.Option;
@@ -230,7 +231,7 @@
     sparkConf.set("spark.hadoop.mapred.output.compression.type", "BLOCK");
 
     additionalConfigs.forEach(sparkConf::set);
-    return HoodieWriteClient.registerClasses(sparkConf);
+    return SparkRDDWriteClient.registerClasses(sparkConf);
   }
 
   public static JavaSparkContext buildSparkContext(String appName, String defaultMaster, Map<String, String> configs) {
@@ -260,8 +261,8 @@
    * @param schemaStr   Schema
    * @param parallelism Parallelism
    */
-  public static HoodieWriteClient createHoodieClient(JavaSparkContext jsc, String basePath, String schemaStr,
-                                                     int parallelism, Option<String> compactionStrategyClass, TypedProperties properties) {
+  public static SparkRDDWriteClient createHoodieClient(JavaSparkContext jsc, String basePath, String schemaStr,
+                                                             int parallelism, Option<String> compactionStrategyClass, TypedProperties properties) {
     HoodieCompactionConfig compactionConfig = compactionStrategyClass
         .map(strategy -> HoodieCompactionConfig.newBuilder().withInlineCompaction(false)
             .withCompactionStrategy(ReflectionUtils.loadClass(strategy)).build())
@@ -274,7 +275,7 @@
             .withSchema(schemaStr).combineInput(true, true).withCompactionConfig(compactionConfig)
             .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
             .withProps(properties).build();
-    return new HoodieWriteClient(jsc, config);
+    return new SparkRDDWriteClient(new HoodieSparkEngineContext(jsc), config);
   }
 
   public static int handleErrors(JavaSparkContext jsc, String instantTime, JavaRDD<WriteStatus> writeResponse) {
diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/BootstrapExecutor.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/BootstrapExecutor.java
index a137cac..c7974b3 100644
--- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/BootstrapExecutor.java
+++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/BootstrapExecutor.java
@@ -20,7 +20,8 @@
 
 import org.apache.hudi.DataSourceUtils;
 import org.apache.hudi.DataSourceWriteOptions;
-import org.apache.hudi.client.HoodieWriteClient;
+import org.apache.hudi.client.SparkRDDWriteClient;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.config.TypedProperties;
 import org.apache.hudi.common.model.HoodieTableType;
 import org.apache.hudi.common.table.HoodieTableConfig;
@@ -137,7 +138,7 @@
    */
   public void execute() throws IOException {
     initializeTable();
-    HoodieWriteClient bootstrapClient = new HoodieWriteClient(jssc, bootstrapConfig, true);
+    SparkRDDWriteClient bootstrapClient = new SparkRDDWriteClient(new HoodieSparkEngineContext(jssc), bootstrapConfig, true);
 
     try {
       HashMap<String, String> checkpointCommitMetadata = new HashMap<>();
diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java
index afe6862..a8d2ac10 100644
--- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java
+++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java
@@ -21,8 +21,9 @@
 import org.apache.hudi.AvroConversionUtils;
 import org.apache.hudi.DataSourceUtils;
 import org.apache.hudi.avro.HoodieAvroUtils;
-import org.apache.hudi.client.HoodieWriteClient;
+import org.apache.hudi.client.SparkRDDWriteClient;
 import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.config.TypedProperties;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.HoodieCommitMetadata;
@@ -158,7 +159,7 @@
   /**
    * Callback when write client is instantiated.
    */
-  private transient Function<HoodieWriteClient, Boolean> onInitializingHoodieWriteClient;
+  private transient Function<SparkRDDWriteClient, Boolean> onInitializingHoodieWriteClient;
 
   /**
    * Timeline with completed commits.
@@ -168,13 +169,13 @@
   /**
    * Write Client.
    */
-  private transient HoodieWriteClient writeClient;
+  private transient SparkRDDWriteClient writeClient;
 
   private transient HoodieDeltaStreamerMetrics metrics;
 
   public DeltaSync(HoodieDeltaStreamer.Config cfg, SparkSession sparkSession, SchemaProvider schemaProvider,
                    TypedProperties props, JavaSparkContext jssc, FileSystem fs, Configuration conf,
-                   Function<HoodieWriteClient, Boolean> onInitializingHoodieWriteClient) throws IOException {
+                   Function<SparkRDDWriteClient, Boolean> onInitializingHoodieWriteClient) throws IOException {
 
     this.cfg = cfg;
     this.jssc = jssc;
@@ -543,7 +544,7 @@
     if ((null != schemaProvider) && (null == writeClient)) {
       registerAvroSchemas(schemaProvider);
       HoodieWriteConfig hoodieCfg = getHoodieClientConfig(schemaProvider);
-      writeClient = new HoodieWriteClient<>(jssc, hoodieCfg, true);
+      writeClient = new SparkRDDWriteClient<>(new HoodieSparkEngineContext(jssc), hoodieCfg, true);
       onInitializingHoodieWriteClient.apply(writeClient);
     }
   }
diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java
index 692422d..5e6e655 100644
--- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java
+++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java
@@ -18,10 +18,12 @@
 
 package org.apache.hudi.utilities.deltastreamer;
 
-import org.apache.hudi.async.AbstractAsyncService;
+import org.apache.hudi.async.HoodieAsyncService;
 import org.apache.hudi.async.AsyncCompactService;
-import org.apache.hudi.client.HoodieWriteClient;
+import org.apache.hudi.async.SparkAsyncCompactService;
+import org.apache.hudi.client.SparkRDDWriteClient;
 import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.bootstrap.index.HFileBootstrapIndex;
 import org.apache.hudi.common.config.TypedProperties;
 import org.apache.hudi.common.fs.FSUtils;
@@ -470,7 +472,7 @@
   /**
    * Syncs data either in single-run or in continuous mode.
    */
-  public static class DeltaSyncService extends AbstractAsyncService {
+  public static class DeltaSyncService extends HoodieAsyncService {
 
     private static final long serialVersionUID = 1L;
     /**
@@ -620,9 +622,9 @@
      * @param writeClient HoodieWriteClient
      * @return
      */
-    protected Boolean onInitializingWriteClient(HoodieWriteClient writeClient) {
+    protected Boolean onInitializingWriteClient(SparkRDDWriteClient writeClient) {
       if (cfg.isAsyncCompactionEnabled()) {
-        asyncCompactService = new AsyncCompactService(jssc, writeClient);
+        asyncCompactService = new SparkAsyncCompactService(new HoodieSparkEngineContext(jssc), writeClient);
         // Enqueue existing pending compactions first
         HoodieTableMetaClient meta =
             new HoodieTableMetaClient(new Configuration(jssc.hadoopConfiguration()), cfg.targetBasePath, true);
diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/perf/TimelineServerPerf.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/perf/TimelineServerPerf.java
index 6aaa6bd..f338e52 100644
--- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/perf/TimelineServerPerf.java
+++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/perf/TimelineServerPerf.java
@@ -18,6 +18,8 @@
 
 package org.apache.hudi.utilities.perf;
 
+import org.apache.hudi.client.common.HoodieEngineContext;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.FileSlice;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
@@ -132,8 +134,9 @@
 
   public List<PerfStats> runLookups(JavaSparkContext jsc, List<String> partitionPaths, SyncableFileSystemView fsView,
       int numIterations, int concurrency) {
-    jsc.setJobGroup(this.getClass().getSimpleName(), "Lookup all performance stats");
-    return jsc.parallelize(partitionPaths, cfg.numExecutors).flatMap(p -> {
+    HoodieEngineContext context = new HoodieSparkEngineContext(jsc);
+    context.setJobStatus(this.getClass().getSimpleName(), "Lookup all performance stats");
+    return context.flatMap(partitionPaths, p -> {
       ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(100);
       final List<PerfStats> result = new ArrayList<>();
       final List<ScheduledFuture<PerfStats>> futures = new ArrayList<>();
@@ -141,7 +144,7 @@
       String fileId = slices.isEmpty() ? "dummyId"
           : slices.get(new Random(Double.doubleToLongBits(Math.random())).nextInt(slices.size())).getFileId();
       IntStream.range(0, concurrency).forEach(i -> futures.add(executor.schedule(() -> runOneRound(fsView, p, fileId,
-              i, numIterations), 0, TimeUnit.NANOSECONDS)));
+          i, numIterations), 0, TimeUnit.NANOSECONDS)));
       futures.forEach(x -> {
         try {
           result.add(x.get());
@@ -151,8 +154,8 @@
       });
       System.out.println("SLICES are=");
       slices.forEach(s -> System.out.println("\t\tFileSlice=" + s));
-      return result.iterator();
-    }).collect();
+      return result.stream();
+    }, cfg.numExecutors);
   }
 
   private static PerfStats runOneRound(SyncableFileSystemView fsView, String partition, String fileId, int id,
diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieSnapshotExporter.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieSnapshotExporter.java
index 39341b2..5f51174 100644
--- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieSnapshotExporter.java
+++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieSnapshotExporter.java
@@ -18,7 +18,7 @@
 
 package org.apache.hudi.utilities.functional;
 
-import org.apache.hudi.client.HoodieWriteClient;
+import org.apache.hudi.client.SparkRDDWriteClient;
 import org.apache.hudi.common.model.HoodieAvroPayload;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieTableType;
@@ -85,7 +85,7 @@
 
     // Prepare data as source Hudi dataset
     HoodieWriteConfig cfg = getHoodieWriteConfig(sourcePath);
-    HoodieWriteClient hdfsWriteClient = new HoodieWriteClient(jsc(), cfg);
+    SparkRDDWriteClient hdfsWriteClient = new SparkRDDWriteClient(context(), cfg);
     hdfsWriteClient.startCommitWithTime(COMMIT_TIME);
     HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(new String[] {PARTITION_PATH});
     List<HoodieRecord> records = dataGen.generateInserts(COMMIT_TIME, NUM_RECORDS);
diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java
index a25d45d..0bbdb23 100644
--- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java
+++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java
@@ -19,6 +19,8 @@
 package org.apache.hudi.utilities.testutils;
 
 import java.io.FileInputStream;
+
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.common.config.TypedProperties;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieTableType;
@@ -84,6 +86,7 @@
   protected static MiniDFSCluster dfsCluster;
   protected static DistributedFileSystem dfs;
   protected transient JavaSparkContext jsc = null;
+  protected transient HoodieSparkEngineContext context = null;
   protected transient SparkSession sparkSession = null;
   protected transient SQLContext sqlContext;
   protected static HiveServer2 hiveServer;
@@ -129,6 +132,7 @@
   public void setup() throws Exception {
     TestDataSource.initDataGen();
     jsc = UtilHelpers.buildSparkContext(this.getClass().getName() + "-hoodie", "local[2]");
+    context = new HoodieSparkEngineContext(jsc);
     sqlContext = new SQLContext(jsc);
     sparkSession = SparkSession.builder().config(jsc.getConf()).getOrCreate();
   }
@@ -139,6 +143,9 @@
     if (jsc != null) {
       jsc.stop();
     }
+    if (context != null) {
+      context = null;
+    }
   }
 
   /**
diff --git a/packaging/hudi-integ-test-bundle/pom.xml b/packaging/hudi-integ-test-bundle/pom.xml
index 48dd169..84285dc 100644
--- a/packaging/hudi-integ-test-bundle/pom.xml
+++ b/packaging/hudi-integ-test-bundle/pom.xml
@@ -69,7 +69,8 @@
                   <include>commons-pool:commons-pool</include>
 
                   <include>org.apache.hudi:hudi-common</include>
-                  <include>org.apache.hudi:hudi-client</include>
+                  <include>org.apache.hudi:hudi-client-common</include>
+                  <include>org.apache.hudi:hudi-spark-client</include>
                   <include>org.apache.hudi:hudi-utilities_${scala.binary.version}</include>
                   <include>org.apache.hudi:hudi-spark_${scala.binary.version}</include>
                   <include>org.apache.hudi:hudi-hive-sync</include>
@@ -431,7 +432,12 @@
 
     <dependency>
       <groupId>org.apache.hudi</groupId>
-      <artifactId>hudi-client</artifactId>
+      <artifactId>hudi-client-common</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hudi</groupId>
+      <artifactId>hudi-spark-client</artifactId>
       <version>${project.version}</version>
     </dependency>
 
@@ -450,7 +456,15 @@
 
     <dependency>
       <groupId>org.apache.hudi</groupId>
-      <artifactId>hudi-client</artifactId>
+      <artifactId>hudi-client-common</artifactId>
+      <version>${project.version}</version>
+      <classifier>tests</classifier>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hudi</groupId>
+      <artifactId>hudi-spark-client</artifactId>
       <version>${project.version}</version>
       <classifier>tests</classifier>
       <type>test-jar</type>
diff --git a/packaging/hudi-spark-bundle/pom.xml b/packaging/hudi-spark-bundle/pom.xml
index 92bf797..2c22ac3 100644
--- a/packaging/hudi-spark-bundle/pom.xml
+++ b/packaging/hudi-spark-bundle/pom.xml
@@ -65,7 +65,8 @@
                 <includes>
                   <include>com.yammer.metrics:metrics-core</include>
                   <include>org.apache.hudi:hudi-common</include>
-                  <include>org.apache.hudi:hudi-client</include>
+                  <include>org.apache.hudi:hudi-client-common</include>
+                  <include>org.apache.hudi:hudi-spark-client</include>
                   <include>org.apache.hudi:hudi-spark_${scala.binary.version}</include>
                   <include>org.apache.hudi:hudi-hive-sync</include>
                   <include>org.apache.hudi:hudi-sync-common</include>
@@ -209,7 +210,12 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hudi</groupId>
-      <artifactId>hudi-client</artifactId>
+      <artifactId>hudi-client-common</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hudi</groupId>
+      <artifactId>hudi-spark-client</artifactId>
       <version>${project.version}</version>
     </dependency>
     <dependency>
diff --git a/packaging/hudi-utilities-bundle/pom.xml b/packaging/hudi-utilities-bundle/pom.xml
index eb4fee7..6bd5fec 100644
--- a/packaging/hudi-utilities-bundle/pom.xml
+++ b/packaging/hudi-utilities-bundle/pom.xml
@@ -66,7 +66,8 @@
               <artifactSet>
                 <includes>
                   <include>org.apache.hudi:hudi-common</include>
-                  <include>org.apache.hudi:hudi-client</include>
+                  <include>org.apache.hudi:hudi-client-common</include>
+                  <include>org.apache.hudi:hudi-spark-client</include>
                   <include>org.apache.hudi:hudi-utilities_${scala.binary.version}</include>
                   <include>org.apache.hudi:hudi-spark_${scala.binary.version}</include>
                   <include>org.apache.hudi:hudi-hive-sync</include>
@@ -211,7 +212,12 @@
     </dependency>
     <dependency>
       <groupId>org.apache.hudi</groupId>
-      <artifactId>hudi-client</artifactId>
+      <artifactId>hudi-client-common</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hudi</groupId>
+      <artifactId>hudi-spark-client</artifactId>
       <version>${project.version}</version>
     </dependency>
     <dependency>
diff --git a/style/checkstyle.xml b/style/checkstyle.xml
index 86575c9..c816abe 100644
--- a/style/checkstyle.xml
+++ b/style/checkstyle.xml
@@ -46,10 +46,12 @@
         <property name="file" value="${checkstyle.suppressions.file}" default="checkstyle-suppressions.xml"/>
         <property name="optional" value="true"/>
     </module>
+    <module name="SuppressWarningsFilter" />
     <module name="TreeWalker">
         <module name="AvoidNestedBlocks">
             <property name="allowInSwitchCase" value="true"/>
         </module>
+        <module name="SuppressWarningsHolder"/>
         <module name="OuterTypeFilename"/>
         <module name="IllegalTokenText">
             <property name="tokens" value="STRING_LITERAL, CHAR_LITERAL"/>