You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by vi...@apache.org on 2019/05/16 20:25:38 UTC

[incubator-hudi] branch master updated: Timeline Service with Incremental View Syncing support

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

vinoth pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-hudi.git


The following commit(s) were added to refs/heads/master by this push:
     new 64fec64  Timeline Service with Incremental View Syncing support
64fec64 is described below

commit 64fec640975b42b0645c75416eb8bbf0b8092c66
Author: Balaji Varadarajan <va...@uber.com>
AuthorDate: Tue Feb 12 21:29:14 2019 -0800

    Timeline Service with Incremental View Syncing support
---
 docker/demo/config/base.properties                 |   2 +
 .../uber/hoodie/cli/commands/CommitsCommand.java   |   7 +-
 .../hoodie/cli/commands/CompactionCommand.java     |   2 +-
 .../hoodie/cli/commands/FileSystemViewCommand.java |  21 +-
 .../com/uber/hoodie/cli/commands/SparkMain.java    |   6 +-
 hoodie-client/pom.xml                              |   5 +
 .../java/com/uber/hoodie/AbstractHoodieClient.java | 100 +++
 .../com/uber/hoodie/CompactionAdminClient.java     |  21 +-
 .../java/com/uber/hoodie/HoodieReadClient.java     |  12 +-
 .../java/com/uber/hoodie/HoodieWriteClient.java    |  29 +-
 .../client/embedded/EmbeddedTimelineService.java   | 100 +++
 .../com/uber/hoodie/config/HoodieWriteConfig.java  |  54 +-
 .../uber/hoodie/index/bloom/HoodieBloomIndex.java  |   3 +-
 .../com/uber/hoodie/io/HoodieAppendHandle.java     |  18 +-
 .../java/com/uber/hoodie/io/HoodieCleanHelper.java |  45 +-
 .../com/uber/hoodie/io/HoodieCreateHandle.java     |   7 +-
 .../java/com/uber/hoodie/io/HoodieIOHandle.java    |   5 +-
 .../java/com/uber/hoodie/io/HoodieMergeHandle.java |  18 +-
 .../io/compact/HoodieRealtimeTableCompactor.java   |  11 +-
 .../io/compact/strategy/CompactionStrategy.java    |   4 +-
 .../LogFileSizeBasedCompactionStrategy.java        |   2 +-
 .../uber/hoodie/table/HoodieCopyOnWriteTable.java  |   7 +-
 .../uber/hoodie/table/HoodieMergeOnReadTable.java  |  12 +-
 .../java/com/uber/hoodie/table/HoodieTable.java    |  28 +-
 .../java/com/uber/hoodie/TestAsyncCompaction.java  |  35 +-
 ...tAsyncCompactionWithEmbeddedTimelineServer.java |  56 ++
 .../src/test/java/com/uber/hoodie/TestCleaner.java |   6 +-
 .../TestCleanerWithEmbeddedTimelineServer.java     |  45 ++
 .../java/com/uber/hoodie/TestClientRollback.java   |   2 +-
 .../java/com/uber/hoodie/TestHoodieClientBase.java |  24 +
 .../TestHoodieClientOnCopyOnWriteStorage.java      |  17 +-
 .../TestHoodieReadClientWithEmbeddedServer.java    |  45 ++
 .../TestHoodieWriteClientWithEmbeddedServer.java   |  55 ++
 .../src/test/java/com/uber/hoodie/TestMultiFS.java |  22 +-
 .../uber/hoodie/TestMultiFSWithEmbeddedServer.java |  36 +
 .../hoodie/io/strategy/TestHoodieDataFile.java     |   2 +-
 .../uber/hoodie/io/strategy/TestHoodieLogFile.java |   7 +-
 .../uber/hoodie/table/TestMergeOnReadTable.java    |  54 +-
 .../TestMergeonReadTableWithEmbeddedServer.java    |  56 ++
 hoodie-common/pom.xml                              |  19 +-
 .../src/main/avro/HoodieCommitMetadata.avsc        |   5 +
 .../hoodie/common/model/CompactionOperation.java   |  69 +-
 .../com/uber/hoodie/common/model/FileSlice.java    |  31 +-
 .../uber/hoodie/common/model/HoodieDataFile.java   |  63 +-
 .../uber/hoodie/common/model/HoodieFileGroup.java  |  25 +-
 .../uber/hoodie/common/model/HoodieLogFile.java    |  74 +-
 .../uber/hoodie/common/model/HoodieWriteStat.java  |  13 +
 .../hoodie/common/table/HoodieTableConfig.java     |   4 +
 .../hoodie/common/table/HoodieTableMetaClient.java |  26 +
 .../uber/hoodie/common/table/HoodieTimeline.java   |  15 +
 .../common/table/SyncableFileSystemView.java       |  44 ++
 .../hoodie/common/table/TableFileSystemView.java   |  68 +-
 .../table/timeline/HoodieActiveTimeline.java       |  18 +-
 .../table/timeline/HoodieArchivedTimeline.java     |   4 +-
 .../table/timeline/HoodieDefaultTimeline.java      |  39 +-
 .../common/table/timeline/HoodieInstant.java       |   4 +-
 .../common/table/timeline/dto/CompactionOpDTO.java |  75 ++
 .../common/table/timeline/dto/DataFileDTO.java     |  60 ++
 .../common/table/timeline/dto/FSPermissionDTO.java |  61 ++
 .../common/table/timeline/dto/FileGroupDTO.java    |  56 ++
 .../common/table/timeline/dto/FilePathDTO.java     |  51 ++
 .../common/table/timeline/dto/FileSliceDTO.java    |  55 ++
 .../common/table/timeline/dto/FileStatusDTO.java   |  97 +++
 .../common/table/timeline/dto/InstantDTO.java      |  53 ++
 .../common/table/timeline/dto/LogFileDTO.java      |  48 ++
 .../common/table/timeline/dto/TimelineDTO.java     |  44 ++
 .../table/view/AbstractTableFileSystemView.java    | 823 +++++++++++++++++++++
 .../common/table/view/FileSystemViewManager.java   | 216 ++++++
 .../table/view/FileSystemViewStorageConfig.java    | 197 +++++
 .../table/view/FileSystemViewStorageType.java      |  34 +
 .../table/view/HoodieTableFileSystemView.java      | 432 +++--------
 .../IncrementalTimelineSyncFileSystemView.java     | 338 +++++++++
 .../table/view/PriorityBasedFileSystemView.java    | 238 ++++++
 .../view/RemoteHoodieTableFileSystemView.java      | 469 ++++++++++++
 .../table/view/RocksDbBasedFileSystemView.java     | 341 +++++++++
 .../view/SpillableMapBasedFileSystemView.java      | 109 +++
 .../uber/hoodie/common/util/CompactionUtils.java   |  25 +-
 .../com/uber/hoodie/common/util/Functions.java     |  38 +
 .../com/uber/hoodie/common/util/NetworkUtils.java  |  42 ++
 .../java/com/uber/hoodie/common/util/Option.java   | 312 ++++++++
 .../com/uber/hoodie/common/util/RocksDBDAO.java    | 389 ++++++++++
 .../hoodie/common/util/RocksDBSchemaHelper.java    | 124 ++++
 .../com/uber/hoodie/common/util/StringUtils.java   |   4 +
 .../hoodie/common/util/TimelineDiffHelper.java     | 134 ++++
 .../common/util/collection/DiskBasedMap.java       |  90 ++-
 .../util/collection/ExternalSpillableMap.java      |  69 +-
 .../common/util/collection/LazyFileIterable.java   |  60 +-
 .../uber/hoodie/config/DefaultHoodieConfig.java    |   0
 .../hoodie/exception/HoodieRemoteException.java    |  27 +
 .../uber/hoodie/common/model/HoodieTestUtils.java  |   9 +-
 .../common/table/string/MockHoodieTimeline.java    |   4 +-
 .../table/view/HoodieTableFileSystemViewTest.java  | 406 ++++++----
 .../table/view/IncrementalFSViewSyncTest.java      | 783 ++++++++++++++++++++
 .../RocksDBBasedIncrementalFSViewSyncTest.java     |  33 +
 .../table/view/RocksDbBasedFileSystemViewTest.java |  30 +
 .../view/SpillableMapBasedFileSystemViewTest.java  |  29 +
 ...SpillableMapBasedIncrementalFSViewSyncTest.java |  30 +
 .../hoodie/common/util/CompactionTestUtils.java    |   2 +-
 .../hoodie/common/util/TestCompactionUtils.java    |   8 +-
 .../hoodie/common/util/TestRocksDBManager.java     | 183 +++++
 {hoodie-client => hoodie-timeline-service}/pom.xml | 165 +++--
 hoodie-timeline-service/run_server.sh              |  14 +
 .../timeline/service/FileSystemViewHandler.java    | 351 +++++++++
 .../hoodie/timeline/service/TimelineService.java   | 161 ++++
 .../timeline/service/handlers/DataFileHandler.java |  76 ++
 .../service/handlers/FileSliceHandler.java         |  92 +++
 .../hoodie/timeline/service/handlers/Handler.java  |  35 +
 .../timeline/service/handlers/TimelineHandler.java |  46 ++
 .../view/RemoteHoodieTableFileSystemViewTest.java  |  57 ++
 .../test/resources/log4j-surefire-quiet.properties |  21 +
 .../src/test/resources/log4j-surefire.properties   |  25 +
 hoodie-utilities/pom.xml                           |  40 +-
 .../hoodie/utilities/HoodieWithTimelineServer.java | 129 ++++
 .../hoodie/utilities/perf/TimelineServerPerf.java  | 308 ++++++++
 packaging/hoodie-hadoop-mr-bundle/pom.xml          |  14 +-
 packaging/hoodie-spark-bundle/pom.xml              |   4 +
 pom.xml                                            |  28 +-
 117 files changed, 8769 insertions(+), 927 deletions(-)

diff --git a/docker/demo/config/base.properties b/docker/demo/config/base.properties
index 9075810..b11c114 100644
--- a/docker/demo/config/base.properties
+++ b/docker/demo/config/base.properties
@@ -19,3 +19,5 @@
 hoodie.upsert.shuffle.parallelism=2
 hoodie.insert.shuffle.parallelism=2
 hoodie.bulkinsert.shuffle.parallelism=2
+hoodie.embed.timeline.server=true
+hoodie.filesystem.view.type=EMBEDDED_KV_STORE
diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CommitsCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CommitsCommand.java
index a3c201c..8e2b696 100644
--- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CommitsCommand.java
+++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CommitsCommand.java
@@ -232,7 +232,9 @@ public class CommitsCommand implements CommandMarker {
       for (HoodieWriteStat stat : stats) {
         rows.add(new Comparable[]{path, stat.getFileId(), stat.getPrevCommit(), stat.getNumUpdateWrites(),
             stat.getNumWrites(), stat.getTotalWriteBytes(),
-            stat.getTotalWriteErrors()});
+            stat.getTotalWriteErrors(),
+            stat.getFileSizeInBytes()
+        });
       }
     }
 
@@ -243,7 +245,8 @@ public class CommitsCommand implements CommandMarker {
         .addTableHeaderField("Total Records Updated")
         .addTableHeaderField("Total Records Written")
         .addTableHeaderField("Total Bytes Written")
-        .addTableHeaderField("Total Errors");
+        .addTableHeaderField("Total Errors")
+        .addTableHeaderField("File Size");
 
     return HoodiePrintHelper.print(header, new HashMap<>(), sortByField, descending, limit, headerOnly, rows);
   }
diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CompactionCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CompactionCommand.java
index f5e3c89..4c3385a 100644
--- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CompactionCommand.java
+++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CompactionCommand.java
@@ -163,7 +163,7 @@ public class CompactionCommand implements CommandMarker {
             op.getBaseInstantTime(),
             op.getDataFilePath(),
             op.getDeltaFilePaths().size(),
-            op.getMetrics().toString()
+            op.getMetrics() == null ? "" : op.getMetrics().toString()
         });
       }
     }
diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/FileSystemViewCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/FileSystemViewCommand.java
index 5d3d098..92f82d4 100644
--- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/FileSystemViewCommand.java
+++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/FileSystemViewCommand.java
@@ -84,10 +84,8 @@ public class FileSystemViewCommand implements CommandMarker {
       row[idx++] = fs.getDataFile().isPresent() ? fs.getDataFile().get().getFileSize() : -1;
       if (!readOptimizedOnly) {
         row[idx++] = fs.getLogFiles().count();
-        row[idx++] = fs.getLogFiles().filter(lf -> lf.getFileSize().isPresent())
-            .mapToLong(lf -> lf.getFileSize().get()).sum();
-        row[idx++] = fs.getLogFiles().filter(lf -> lf.getFileSize().isPresent())
-            .collect(Collectors.toList()).toString();
+        row[idx++] = fs.getLogFiles().mapToLong(lf -> lf.getFileSize()).sum();
+        row[idx++] = fs.getLogFiles().collect(Collectors.toList()).toString();
       }
       rows.add(row);
     }));
@@ -162,16 +160,15 @@ public class FileSystemViewCommand implements CommandMarker {
 
       if (!readOptimizedOnly) {
         row[idx++] = fs.getLogFiles().count();
-        row[idx++] = fs.getLogFiles().filter(lf -> lf.getFileSize().isPresent())
-            .mapToLong(lf -> lf.getFileSize().get()).sum();
-        long logFilesScheduledForCompactionTotalSize = fs.getLogFiles().filter(lf -> lf.getFileSize().isPresent())
+        row[idx++] = fs.getLogFiles().mapToLong(lf -> lf.getFileSize()).sum();
+        long logFilesScheduledForCompactionTotalSize = fs.getLogFiles()
             .filter(lf -> lf.getBaseCommitTime().equals(fs.getBaseInstantTime()))
-            .mapToLong(lf -> lf.getFileSize().get()).sum();
+            .mapToLong(lf -> lf.getFileSize()).sum();
         row[idx++] = logFilesScheduledForCompactionTotalSize;
 
-        long logFilesUnscheduledTotalSize = fs.getLogFiles().filter(lf -> lf.getFileSize().isPresent())
+        long logFilesUnscheduledTotalSize = fs.getLogFiles()
             .filter(lf -> !lf.getBaseCommitTime().equals(fs.getBaseInstantTime()))
-            .mapToLong(lf -> lf.getFileSize().get()).sum();
+            .mapToLong(lf -> lf.getFileSize()).sum();
         row[idx++] = logFilesUnscheduledTotalSize;
 
         double logSelectedForCompactionToBaseRatio =
@@ -181,10 +178,10 @@ public class FileSystemViewCommand implements CommandMarker {
             dataFileSize > 0 ? logFilesUnscheduledTotalSize / (dataFileSize * 1.0) : -1;
         row[idx++] = logUnscheduledToBaseRatio;
 
-        row[idx++] = fs.getLogFiles().filter(lf -> lf.getFileSize().isPresent())
+        row[idx++] = fs.getLogFiles()
             .filter(lf -> lf.getBaseCommitTime().equals(fs.getBaseInstantTime()))
             .collect(Collectors.toList()).toString();
-        row[idx++] = fs.getLogFiles().filter(lf -> lf.getFileSize().isPresent())
+        row[idx++] = fs.getLogFiles()
             .filter(lf -> !lf.getBaseCommitTime().equals(fs.getBaseInstantTime()))
             .collect(Collectors.toList()).toString();
       }
diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SparkMain.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SparkMain.java
index 165ae67..d42ac12 100644
--- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SparkMain.java
+++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/SparkMain.java
@@ -78,17 +78,17 @@ public class SparkMain {
         break;
       case COMPACT_SCHEDULE:
         assert (args.length == 5);
-        returnCode = compact(jsc, args[1], args[2], args[3],  1,
+        returnCode = compact(jsc, args[1], args[2], args[3], 1,
             "", args[4], 0, true);
         break;
       case COMPACT_VALIDATE:
         assert (args.length == 7);
-        doCompactValidate(jsc, args[1], args[2], args[3],  Integer.parseInt(args[4]), args[5], args[6]);
+        doCompactValidate(jsc, args[1], args[2], args[3], Integer.parseInt(args[4]), args[5], args[6]);
         returnCode = 0;
         break;
       case COMPACT_REPAIR:
         assert (args.length == 8);
-        doCompactRepair(jsc, args[1], args[2], args[3],  Integer.parseInt(args[4]), args[5], args[6],
+        doCompactRepair(jsc, args[1], args[2], args[3], Integer.parseInt(args[4]), args[5], args[6],
             Boolean.valueOf(args[7]));
         returnCode = 0;
         break;
diff --git a/hoodie-client/pom.xml b/hoodie-client/pom.xml
index 57a4ad8..8f61b3c 100644
--- a/hoodie-client/pom.xml
+++ b/hoodie-client/pom.xml
@@ -69,6 +69,11 @@
       <version>${project.version}</version>
     </dependency>
     <dependency>
+      <groupId>com.uber.hoodie</groupId>
+      <artifactId>hoodie-timeline-service</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-hdfs</artifactId>
       <classifier>tests</classifier>
diff --git a/hoodie-client/src/main/java/com/uber/hoodie/AbstractHoodieClient.java b/hoodie-client/src/main/java/com/uber/hoodie/AbstractHoodieClient.java
new file mode 100644
index 0000000..1df574e
--- /dev/null
+++ b/hoodie-client/src/main/java/com/uber/hoodie/AbstractHoodieClient.java
@@ -0,0 +1,100 @@
+/*
+ *  Copyright (c) 2018 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  You may obtain a copy of the License at
+ *
+ *           http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package com.uber.hoodie;
+
+import com.uber.hoodie.client.embedded.EmbeddedTimelineService;
+import com.uber.hoodie.common.util.FSUtils;
+import com.uber.hoodie.config.HoodieWriteConfig;
+import java.io.IOException;
+import java.io.Serializable;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.api.java.JavaSparkContext;
+
+/**
+ * Abstract class taking care of holding common member variables (FileSystem, SparkContext, HoodieConfigs)
+ * Also, manages embedded timeline-server if enabled.
+ */
+public abstract class AbstractHoodieClient implements Serializable {
+
+  private static final Logger logger = LogManager.getLogger(AbstractHoodieClient.class);
+
+  protected final transient FileSystem fs;
+  protected final transient JavaSparkContext jsc;
+  protected final HoodieWriteConfig config;
+  protected final String basePath;
+
+  /**
+   * Timeline Server has the same lifetime as that of Client.
+   * Any operations done on the same timeline service will be able to take advantage
+   * of the cached file-system view. New completed actions will be synced automatically
+   * in an incremental fashion.
+   */
+  private transient EmbeddedTimelineService timelineServer;
+
+  protected AbstractHoodieClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig) {
+    this.fs = FSUtils.getFs(clientConfig.getBasePath(), jsc.hadoopConfiguration());
+    this.jsc = jsc;
+    this.basePath = clientConfig.getBasePath();
+    this.config = clientConfig;
+    startEmbeddedServerView();
+  }
+
+  /**
+   * Releases any resources used by the client.
+   */
+  public void close() {
+    stopEmbeddedServerView(true);
+  }
+
+  private synchronized void stopEmbeddedServerView(boolean resetViewStorageConfig) {
+    if (timelineServer != null) {
+      logger.info("Stopping Timeline service !!");
+      timelineServer.stop();
+      timelineServer = null;
+      // Reset Storage Config to Client specified config
+      if (resetViewStorageConfig) {
+        config.resetViewStorageConfig();
+      }
+    }
+  }
+
+  private synchronized void startEmbeddedServerView() {
+    if (config.isEmbeddedTimelineServerEnabled()) {
+      if (timelineServer == null) {
+        // Run Embedded Timeline Server
+        logger.info("Starting Timeline service !!");
+        timelineServer = new EmbeddedTimelineService(jsc.hadoopConfiguration(), jsc.getConf(),
+            config.getClientSpecifiedViewStorageConfig());
+        try {
+          timelineServer.startServer();
+          // Allow executor to find this newly instantiated timeline service
+          config.setViewStorageConfig(timelineServer.getRemoteFileSystemViewConfig());
+        } catch (IOException e) {
+          logger.warn("Unable to start timeline service. Proceeding as if embedded server is disabled", e);
+          stopEmbeddedServerView(false);
+        }
+      } else {
+        logger.info("Timeline Server already running. Not restarting the service");
+      }
+    } else {
+      logger.info("Embedded Timeline Server is disabled. Not starting timeline service");
+    }
+  }
+}
diff --git a/hoodie-client/src/main/java/com/uber/hoodie/CompactionAdminClient.java b/hoodie-client/src/main/java/com/uber/hoodie/CompactionAdminClient.java
index 715bb3d..547a0a4 100644
--- a/hoodie-client/src/main/java/com/uber/hoodie/CompactionAdminClient.java
+++ b/hoodie-client/src/main/java/com/uber/hoodie/CompactionAdminClient.java
@@ -36,6 +36,7 @@ import com.uber.hoodie.common.util.AvroUtils;
 import com.uber.hoodie.common.util.CompactionUtils;
 import com.uber.hoodie.common.util.FSUtils;
 import com.uber.hoodie.common.util.collection.Pair;
+import com.uber.hoodie.config.HoodieWriteConfig;
 import com.uber.hoodie.exception.HoodieException;
 import com.uber.hoodie.exception.HoodieIOException;
 import com.uber.hoodie.func.OperationResult;
@@ -56,16 +57,12 @@ import org.apache.spark.api.java.JavaSparkContext;
 /**
  * Client to perform admin operations related to compaction
  */
-public class CompactionAdminClient implements Serializable {
+public class CompactionAdminClient extends AbstractHoodieClient {
 
   private static Logger log = LogManager.getLogger(CompactionAdminClient.class);
 
-  private final transient JavaSparkContext jsc;
-  private final String basePath;
-
   public CompactionAdminClient(JavaSparkContext jsc, String basePath) {
-    this.jsc = jsc;
-    this.basePath = basePath;
+    super(jsc, HoodieWriteConfig.newBuilder().withPath(basePath).build());
   }
 
   /**
@@ -123,19 +120,17 @@ public class CompactionAdminClient implements Serializable {
     // Only if all operations are successfully executed
     if (!dryRun && allSuccess.isPresent() && allSuccess.get()) {
       // Overwrite compaction request with empty compaction operations
-      HoodieCompactionPlan plan = CompactionUtils.getCompactionPlan(metaClient, compactionInstant);
-      HoodieCompactionPlan newPlan =
-          HoodieCompactionPlan.newBuilder().setOperations(new ArrayList<>()).setExtraMetadata(plan.getExtraMetadata())
-              .build();
       HoodieInstant inflight = new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, compactionInstant);
       Path inflightPath = new Path(metaClient.getMetaPath(), inflight.getFileName());
       if (metaClient.getFs().exists(inflightPath)) {
         // We need to rollback data-files because of this inflight compaction before unscheduling
         throw new IllegalStateException("Please rollback the inflight compaction before unscheduling");
       }
-      metaClient.getActiveTimeline().saveToCompactionRequested(
-          new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, compactionInstant),
-          AvroUtils.serializeCompactionPlan(newPlan));
+      // Leave the trace in aux folder but delete from metapath.
+      // TODO: Add a rollback instant but for compaction
+      HoodieInstant instant = new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, compactionInstant);
+      boolean deleted = metaClient.getFs().delete(new Path(metaClient.getMetaPath(), instant.getFileName()), false);
+      Preconditions.checkArgument(deleted, "Unable to delete compaction instant.");
     }
     return res;
   }
diff --git a/hoodie-client/src/main/java/com/uber/hoodie/HoodieReadClient.java b/hoodie-client/src/main/java/com/uber/hoodie/HoodieReadClient.java
index 94a48b1..1aa3aa3 100644
--- a/hoodie-client/src/main/java/com/uber/hoodie/HoodieReadClient.java
+++ b/hoodie-client/src/main/java/com/uber/hoodie/HoodieReadClient.java
@@ -24,19 +24,16 @@ import com.uber.hoodie.common.model.HoodieRecordPayload;
 import com.uber.hoodie.common.table.HoodieTableMetaClient;
 import com.uber.hoodie.common.table.HoodieTimeline;
 import com.uber.hoodie.common.util.CompactionUtils;
-import com.uber.hoodie.common.util.FSUtils;
 import com.uber.hoodie.common.util.collection.Pair;
 import com.uber.hoodie.config.HoodieIndexConfig;
 import com.uber.hoodie.config.HoodieWriteConfig;
 import com.uber.hoodie.exception.HoodieIndexException;
 import com.uber.hoodie.index.HoodieIndex;
 import com.uber.hoodie.table.HoodieTable;
-import java.io.Serializable;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 import java.util.stream.Collectors;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
 import org.apache.spark.SparkConf;
@@ -52,13 +49,10 @@ import scala.Tuple2;
 /**
  * Provides an RDD based API for accessing/filtering Hoodie tables, based on keys.
  */
-public class HoodieReadClient<T extends HoodieRecordPayload> implements Serializable {
+public class HoodieReadClient<T extends HoodieRecordPayload> extends AbstractHoodieClient {
 
   private static final Logger logger = LogManager.getLogger(HoodieReadClient.class);
 
-  private final transient JavaSparkContext jsc;
-
-  private final transient FileSystem fs;
   /**
    * 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 dataset. Until, then just always assume a
@@ -94,9 +88,8 @@ public class HoodieReadClient<T extends HoodieRecordPayload> implements Serializ
    * @param clientConfig instance of HoodieWriteConfig
    */
   public HoodieReadClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig) {
+    super(jsc, clientConfig);
     final String basePath = clientConfig.getBasePath();
-    this.jsc = jsc;
-    this.fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration());
     // Create a Hoodie table which encapsulated the commits and files visible
     HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true);
     this.hoodieTable = HoodieTable
@@ -130,7 +123,6 @@ public class HoodieReadClient<T extends HoodieRecordPayload> implements Serializ
    * @return a dataframe
    */
   public Dataset<Row> read(JavaRDD<HoodieKey> hoodieKeys, int parallelism) throws Exception {
-
     assertSqlContext();
     JavaPairRDD<HoodieKey, Optional<String>> keyToFileRDD = index
         .fetchRecordLocation(hoodieKeys, jsc, hoodieTable);
diff --git a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java
index ff46794..92bac7a 100644
--- a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java
+++ b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java
@@ -62,7 +62,6 @@ import com.uber.hoodie.table.UserDefinedBulkInsertPartitioner;
 import com.uber.hoodie.table.WorkloadProfile;
 import com.uber.hoodie.table.WorkloadStat;
 import java.io.IOException;
-import java.io.Serializable;
 import java.nio.charset.StandardCharsets;
 import java.text.ParseException;
 import java.util.Arrays;
@@ -73,7 +72,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Optional;
 import java.util.stream.Collectors;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
 import org.apache.spark.Partitioner;
@@ -92,12 +90,9 @@ import scala.Tuple2;
  * Note that, at any given time, there can only be one Spark job performing these operatons on a
  * Hoodie dataset.
  */
-public class HoodieWriteClient<T extends HoodieRecordPayload> implements Serializable {
+public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHoodieClient {
 
   private static Logger logger = LogManager.getLogger(HoodieWriteClient.class);
-  private final transient FileSystem fs;
-  private final transient JavaSparkContext jsc;
-  private final HoodieWriteConfig config;
   private final boolean rollbackInFlight;
   private final transient HoodieMetrics metrics;
   private final transient HoodieIndex<T> index;
@@ -126,9 +121,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
   @VisibleForTesting
   HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig,
       boolean rollbackInFlight, HoodieIndex index) {
-    this.fs = FSUtils.getFs(clientConfig.getBasePath(), jsc.hadoopConfiguration());
-    this.jsc = jsc;
-    this.config = clientConfig;
+    super(jsc, clientConfig);
     this.index = index;
     this.metrics = new HoodieMetrics(config, config.getTableName());
     this.rollbackInFlight = rollbackInFlight;
@@ -160,7 +153,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
    * Upserts a bunch of new records into the Hoodie table, at the supplied commitTime
    */
   public JavaRDD<WriteStatus> upsert(JavaRDD<HoodieRecord<T>> records, final String commitTime) {
-    HoodieTable<T> table = getTableAndInitCtx();
+    HoodieTable<T> table = getTableAndInitCtx(records);
     try {
       // De-dupe/merge if needed
       JavaRDD<HoodieRecord<T>> dedupedRecords = combineOnCondition(
@@ -189,7 +182,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
    */
   public JavaRDD<WriteStatus> upsertPreppedRecords(JavaRDD<HoodieRecord<T>> preppedRecords,
       final String commitTime) {
-    HoodieTable<T> table = getTableAndInitCtx();
+    HoodieTable<T> table = getTableAndInitCtx(preppedRecords);
     try {
       return upsertRecordsInternal(preppedRecords, commitTime, table, true);
     } catch (Throwable e) {
@@ -213,7 +206,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
    * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
    */
   public JavaRDD<WriteStatus> insert(JavaRDD<HoodieRecord<T>> records, final String commitTime) {
-    HoodieTable<T> table = getTableAndInitCtx();
+    HoodieTable<T> table = getTableAndInitCtx(records);
     try {
       // De-dupe/merge if needed
       JavaRDD<HoodieRecord<T>> dedupedRecords = combineOnCondition(
@@ -241,7 +234,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
    */
   public JavaRDD<WriteStatus> insertPreppedRecords(JavaRDD<HoodieRecord<T>> preppedRecords,
       final String commitTime) {
-    HoodieTable<T> table = getTableAndInitCtx();
+    HoodieTable<T> table = getTableAndInitCtx(preppedRecords);
     try {
       return upsertRecordsInternal(preppedRecords, commitTime, table, false);
     } catch (Throwable e) {
@@ -290,7 +283,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
    */
   public JavaRDD<WriteStatus> bulkInsert(JavaRDD<HoodieRecord<T>> records, final String commitTime,
       Option<UserDefinedBulkInsertPartitioner> bulkInsertPartitioner) {
-    HoodieTable<T> table = getTableAndInitCtx();
+    HoodieTable<T> table = getTableAndInitCtx(records);
     try {
       // De-dupe/merge if needed
       JavaRDD<HoodieRecord<T>> dedupedRecords = combineOnCondition(
@@ -324,7 +317,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
    */
   public JavaRDD<WriteStatus> bulkInsertPreppedRecords(JavaRDD<HoodieRecord<T>> preppedRecords,
       final String commitTime, Option<UserDefinedBulkInsertPartitioner> bulkInsertPartitioner) {
-    HoodieTable<T> table = getTableAndInitCtx();
+    HoodieTable<T> table = getTableAndInitCtx(preppedRecords);
     try {
       return bulkInsertInternal(preppedRecords, commitTime, table, bulkInsertPartitioner);
     } catch (Throwable e) {
@@ -977,7 +970,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
    * Releases any resources used by the client.
    */
   public void close() {
-    // UNDER CONSTRUCTION
+    // Stop timeline-server if running
+    super.close();
   }
 
   /**
@@ -1189,8 +1183,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
     }
   }
 
-  private HoodieTable getTableAndInitCtx() {
-    // Create a Hoodie table which encapsulated the commits and files visible
+  private HoodieTable getTableAndInitCtx(JavaRDD<HoodieRecord<T>> records) {
     // Create a Hoodie table which encapsulated the commits and files visible
     HoodieTable table = HoodieTable.getHoodieTable(
         new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc);
diff --git a/hoodie-client/src/main/java/com/uber/hoodie/client/embedded/EmbeddedTimelineService.java b/hoodie-client/src/main/java/com/uber/hoodie/client/embedded/EmbeddedTimelineService.java
new file mode 100644
index 0000000..cadb674
--- /dev/null
+++ b/hoodie-client/src/main/java/com/uber/hoodie/client/embedded/EmbeddedTimelineService.java
@@ -0,0 +1,100 @@
+/*
+ *  Copyright (c) 2019 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  You may obtain a copy of the License at
+ *
+ *           http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package com.uber.hoodie.client.embedded;
+
+import com.uber.hoodie.common.SerializableConfiguration;
+import com.uber.hoodie.common.table.view.FileSystemViewManager;
+import com.uber.hoodie.common.table.view.FileSystemViewStorageConfig;
+import com.uber.hoodie.common.table.view.FileSystemViewStorageType;
+import com.uber.hoodie.common.util.NetworkUtils;
+import com.uber.hoodie.timeline.service.TimelineService;
+import java.io.IOException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.SparkConf;
+
+/**
+ * Timeline Service that runs as part of write client
+ */
+public class EmbeddedTimelineService {
+
+  private static Logger logger = LogManager.getLogger(EmbeddedTimelineService.class);
+
+  private int serverPort;
+  private String hostAddr;
+  private final SerializableConfiguration hadoopConf;
+  private final FileSystemViewStorageConfig config;
+  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();
+    }
+    this.config = config;
+    this.hadoopConf = new SerializableConfiguration(hadoopConf);
+    this.viewManager = createViewManager();
+  }
+
+  private FileSystemViewManager createViewManager() {
+    // Using passed-in configs to build view storage configs
+    FileSystemViewStorageConfig.Builder builder =
+        FileSystemViewStorageConfig.newBuilder().fromProperties(config.getProps());
+    FileSystemViewStorageType storageType = builder.build().getStorageType();
+    if (storageType.equals(FileSystemViewStorageType.REMOTE_ONLY)
+        || storageType.equals(FileSystemViewStorageType.REMOTE_FIRST)) {
+      // Reset to default if set to Remote
+      builder.withStorageType(FileSystemViewStorageType.MEMORY);
+    }
+    return FileSystemViewManager.createViewManager(hadoopConf, builder.build());
+  }
+
+  public void startServer() throws IOException {
+    server = new TimelineService(0, viewManager);
+    serverPort = server.startService();
+    logger.info("Started embedded timeline server at " + hostAddr + ":" + serverPort);
+  }
+
+  private void setHostAddrFromSparkConf(SparkConf sparkConf) {
+    String hostAddr = sparkConf.get("spark.driver.host", null);
+    if (hostAddr != null) {
+      logger.info("Overriding hostIp to (" + hostAddr + ") found in spark-conf. It was " + this.hostAddr);
+      this.hostAddr = hostAddr;
+    } else {
+      logger.warn("Unable to find driver bind address from spark config");
+    }
+  }
+
+  /**
+   * Retrieves proper view storage configs for remote clients to access this service
+   */
+  public FileSystemViewStorageConfig getRemoteFileSystemViewConfig() {
+    return FileSystemViewStorageConfig.newBuilder().withStorageType(FileSystemViewStorageType.REMOTE_FIRST)
+        .withRemoteServerHost(hostAddr).withRemoteServerPort(serverPort).build();
+  }
+
+  public void stop() {
+    if (null != server) {
+      this.server.close();
+      this.server = null;
+      this.viewManager = null;
+    }
+  }
+}
\ No newline at end of file
diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java
index 0834e1f..e05a435 100644
--- a/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java
+++ b/hoodie-client/src/main/java/com/uber/hoodie/config/HoodieWriteConfig.java
@@ -19,6 +19,7 @@ package com.uber.hoodie.config;
 import com.google.common.base.Preconditions;
 import com.uber.hoodie.WriteStatus;
 import com.uber.hoodie.common.model.HoodieCleaningPolicy;
+import com.uber.hoodie.common.table.view.FileSystemViewStorageConfig;
 import com.uber.hoodie.common.util.ReflectionUtils;
 import com.uber.hoodie.index.HoodieIndex;
 import com.uber.hoodie.io.compact.strategy.CompactionStrategy;
@@ -71,9 +72,20 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
   private static final String DEFAULT_FINALIZE_WRITE_PARALLELISM = DEFAULT_PARALLELISM;
   private static final String CONSISTENCY_CHECK_ENABLED = "hoodie.consistency.check.enabled";
   private static final String DEFAULT_CONSISTENCY_CHECK_ENABLED = "false";
+  private static final String EMBEDDED_TIMELINE_SERVER_ENABLED = "hoodie.embed.timeline.server";
+  private static final String DEFAULT_EMBEDDED_TIMELINE_SERVER_ENABLED = "false";
+
+  // Hoodie Write Client transparently rewrites File System View config when embedded mode is enabled
+  // We keep track of original config and rewritten config
+  private final FileSystemViewStorageConfig clientSpecifiedViewStorageConfig;
+  private FileSystemViewStorageConfig viewStorageConfig;
 
   private HoodieWriteConfig(Properties props) {
     super(props);
+    Properties newProps = new Properties();
+    newProps.putAll(props);
+    this.clientSpecifiedViewStorageConfig = FileSystemViewStorageConfig.newBuilder().fromProperties(newProps).build();
+    this.viewStorageConfig = clientSpecifiedViewStorageConfig;
   }
 
   public static HoodieWriteConfig.Builder newBuilder() {
@@ -157,6 +169,10 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
     return Boolean.parseBoolean(props.getProperty(CONSISTENCY_CHECK_ENABLED));
   }
 
+  public boolean isEmbeddedTimelineServerEnabled() {
+    return Boolean.parseBoolean(props.getProperty(EMBEDDED_TIMELINE_SERVER_ENABLED));
+  }
+
   /**
    * compaction properties
    **/
@@ -434,6 +450,22 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
     return Double.valueOf(props.getProperty(HoodieMemoryConfig.WRITESTATUS_FAILURE_FRACTION_PROP));
   }
 
+  public FileSystemViewStorageConfig getViewStorageConfig() {
+    return viewStorageConfig;
+  }
+
+  public void setViewStorageConfig(FileSystemViewStorageConfig viewStorageConfig) {
+    this.viewStorageConfig = viewStorageConfig;
+  }
+
+  public void resetViewStorageConfig() {
+    this.setViewStorageConfig(getClientSpecifiedViewStorageConfig());
+  }
+
+  public FileSystemViewStorageConfig getClientSpecifiedViewStorageConfig() {
+    return clientSpecifiedViewStorageConfig;
+  }
+
   public static class Builder {
 
     private final Properties props = new Properties();
@@ -442,6 +474,7 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
     private boolean isCompactionConfigSet = false;
     private boolean isMetricsConfigSet = false;
     private boolean isMemoryConfigSet = false;
+    private boolean isViewConfigSet = false;
 
     public Builder fromFile(File propertiesFile) throws IOException {
       FileReader reader = new FileReader(propertiesFile);
@@ -569,6 +602,12 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
       return this;
     }
 
+    public Builder withFileSystemViewConfig(FileSystemViewStorageConfig viewStorageConfig) {
+      props.putAll(viewStorageConfig.getProps());
+      isViewConfigSet = true;
+      return this;
+    }
+
     public Builder withFinalizeWriteParallelism(int parallelism) {
       props.setProperty(FINALIZE_WRITE_PARALLELISM, String.valueOf(parallelism));
       return this;
@@ -579,10 +618,13 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
       return this;
     }
 
+    public Builder withEmbeddedTimelineServerEnabled(boolean enabled) {
+      props.setProperty(EMBEDDED_TIMELINE_SERVER_ENABLED, String.valueOf(enabled));
+      return this;
+    }
+
     public HoodieWriteConfig build() {
-      HoodieWriteConfig config = new HoodieWriteConfig(props);
       // Check for mandatory properties
-      Preconditions.checkArgument(config.getBasePath() != null);
       setDefaultOnCondition(props, !props.containsKey(INSERT_PARALLELISM), INSERT_PARALLELISM,
           DEFAULT_PARALLELISM);
       setDefaultOnCondition(props, !props.containsKey(BULKINSERT_PARALLELISM),
@@ -611,6 +653,8 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
           FINALIZE_WRITE_PARALLELISM, DEFAULT_FINALIZE_WRITE_PARALLELISM);
       setDefaultOnCondition(props, !props.containsKey(CONSISTENCY_CHECK_ENABLED),
           CONSISTENCY_CHECK_ENABLED, DEFAULT_CONSISTENCY_CHECK_ENABLED);
+      setDefaultOnCondition(props, !props.containsKey(EMBEDDED_TIMELINE_SERVER_ENABLED),
+          EMBEDDED_TIMELINE_SERVER_ENABLED, DEFAULT_EMBEDDED_TIMELINE_SERVER_ENABLED);
 
       // Make sure the props is propagated
       setDefaultOnCondition(props, !isIndexConfigSet,
@@ -623,6 +667,12 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
           HoodieMetricsConfig.newBuilder().fromProperties(props).build());
       setDefaultOnCondition(props, !isMemoryConfigSet,
           HoodieMemoryConfig.newBuilder().fromProperties(props).build());
+      setDefaultOnCondition(props, !isViewConfigSet,
+          FileSystemViewStorageConfig.newBuilder().fromProperties(props).build());
+
+      // Build WriteConfig at the end
+      HoodieWriteConfig config = new HoodieWriteConfig(props);
+      Preconditions.checkArgument(config.getBasePath() != null);
       return config;
     }
   }
diff --git a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java
index ccafc56..b5ef59e 100644
--- a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java
+++ b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java
@@ -224,6 +224,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
   @VisibleForTesting
   List<Tuple2<String, BloomIndexFileInfo>> loadInvolvedFiles(List<String> partitions, final JavaSparkContext jsc,
       final HoodieTable hoodieTable) {
+
     // Obtain the latest data files from all the partitions.
     List<Tuple2<String, HoodieDataFile>> dataFilesList = jsc
         .parallelize(partitions, Math.max(partitions.size(), 1)).flatMapToPair(partitionPath -> {
@@ -243,7 +244,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
       return jsc.parallelize(dataFilesList, Math.max(dataFilesList.size(), 1)).mapToPair(ft -> {
         try {
           String[] minMaxKeys = ParquetUtils
-              .readMinMaxRecordKeys(hoodieTable.getHadoopConf(), ft._2().getFileStatus().getPath());
+              .readMinMaxRecordKeys(hoodieTable.getHadoopConf(), new Path(ft._2().getPath()));
           return new Tuple2<>(ft._1(),
               new BloomIndexFileInfo(ft._2().getFileName(), minMaxKeys[0], minMaxKeys[1]));
         } catch (MetadataNotFoundException me) {
diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java
index 17f4d26..1f5e95f 100644
--- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java
+++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java
@@ -26,13 +26,14 @@ import com.uber.hoodie.common.model.HoodieRecord;
 import com.uber.hoodie.common.model.HoodieRecordLocation;
 import com.uber.hoodie.common.model.HoodieRecordPayload;
 import com.uber.hoodie.common.model.HoodieWriteStat.RuntimeStats;
-import com.uber.hoodie.common.table.TableFileSystemView;
+import com.uber.hoodie.common.table.TableFileSystemView.RealtimeView;
 import com.uber.hoodie.common.table.log.HoodieLogFormat;
 import com.uber.hoodie.common.table.log.HoodieLogFormat.Writer;
 import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock;
 import com.uber.hoodie.common.table.log.block.HoodieDeleteBlock;
 import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
 import com.uber.hoodie.common.util.HoodieAvroUtils;
+import com.uber.hoodie.common.util.Option;
 import com.uber.hoodie.config.HoodieWriteConfig;
 import com.uber.hoodie.exception.HoodieAppendException;
 import com.uber.hoodie.exception.HoodieUpsertException;
@@ -67,7 +68,7 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
   private List<IndexedRecord> recordList = new ArrayList<>();
   // Buffer for holding records (to be deleted) in memory before they are flushed to disk
   private List<HoodieKey> keysToDelete = new ArrayList<>();
-  private TableFileSystemView.RealtimeView fileSystemView;
+
   private String partitionPath;
   private Iterator<HoodieRecord<T>> recordItr;
   // Total number of records written during an append
@@ -98,7 +99,6 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
     super(config, commitTime, hoodieTable);
     writeStatus.setStat(new HoodieDeltaWriteStat());
     this.fileId = fileId;
-    this.fileSystemView = hoodieTable.getRTFileSystemView();
     this.recordItr = recordItr;
   }
 
@@ -110,15 +110,15 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
     if (doInit) {
       this.partitionPath = record.getPartitionPath();
       // extract some information from the first record
-      Optional<FileSlice> fileSlice = fileSystemView.getLatestFileSlices(partitionPath)
-          .filter(fileSlice1 -> fileSlice1.getFileId().equals(fileId)).findFirst();
+      RealtimeView rtView = hoodieTable.getRTFileSystemView();
+      Option<FileSlice> fileSlice = rtView.getLatestFileSlice(partitionPath, fileId);
       // Set the base commit time as the current commitTime for new inserts into log files
       String baseInstantTime = commitTime;
       if (fileSlice.isPresent()) {
         baseInstantTime = fileSlice.get().getBaseInstantTime();
       } else {
         // This means there is no base data file, start appending to a new log file
-        fileSlice = Optional.of(new FileSlice(partitionPath, baseInstantTime, this.fileId));
+        fileSlice = Option.of(new FileSlice(partitionPath, baseInstantTime, this.fileId));
         logger.info("New InsertHandle for partition :" + partitionPath);
       }
       writeStatus.getStat().setPrevCommit(baseInstantTime);
@@ -242,6 +242,7 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
     try {
       // flush any remaining records to disk
       doAppend(header);
+      long sizeInBytes = writer.getCurrentSize();
       if (writer != null) {
         writer.close();
       }
@@ -251,6 +252,7 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
       writeStatus.getStat().setNumInserts(insertRecordsWritten);
       writeStatus.getStat().setNumDeletes(recordsDeleted);
       writeStatus.getStat().setTotalWriteBytes(estimatedNumberOfBytesWritten);
+      writeStatus.getStat().setFileSizeInBytes(sizeInBytes);
       writeStatus.getStat().setTotalWriteErrors(writeStatus.getTotalErrorRecords());
       RuntimeStats runtimeStats = new RuntimeStats();
       runtimeStats.setTotalUpsertTime(timer.endTimer());
@@ -266,7 +268,7 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
     return writeStatus;
   }
 
-  private Writer createLogWriter(Optional<FileSlice> fileSlice, String baseCommitTime)
+  private Writer createLogWriter(Option<FileSlice> fileSlice, String baseCommitTime)
       throws IOException, InterruptedException {
     return HoodieLogFormat.newWriterBuilder()
         .onParentPath(new Path(hoodieTable.getMetaClient().getBasePath(), partitionPath))
@@ -305,4 +307,4 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
     }
   }
 
-}
\ No newline at end of file
+}
diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCleanHelper.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCleanHelper.java
index aeb9955..384d5f1 100644
--- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCleanHelper.java
+++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCleanHelper.java
@@ -25,9 +25,8 @@ import com.uber.hoodie.common.model.HoodieFileGroupId;
 import com.uber.hoodie.common.model.HoodieRecordPayload;
 import com.uber.hoodie.common.model.HoodieTableType;
 import com.uber.hoodie.common.table.HoodieTimeline;
-import com.uber.hoodie.common.table.TableFileSystemView;
+import com.uber.hoodie.common.table.SyncableFileSystemView;
 import com.uber.hoodie.common.table.timeline.HoodieInstant;
-import com.uber.hoodie.common.table.view.HoodieTableFileSystemView;
 import com.uber.hoodie.common.util.collection.Pair;
 import com.uber.hoodie.config.HoodieWriteConfig;
 import com.uber.hoodie.table.HoodieTable;
@@ -51,7 +50,7 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
 
   private static Logger logger = LogManager.getLogger(HoodieCleanHelper.class);
 
-  private final TableFileSystemView fileSystemView;
+  private final SyncableFileSystemView fileSystemView;
   private final HoodieTimeline commitTimeline;
   private final Map<HoodieFileGroupId, CompactionOperation> fgIdToPendingCompactionOperations;
   private HoodieTable<T> hoodieTable;
@@ -59,12 +58,13 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
 
   public HoodieCleanHelper(HoodieTable<T> hoodieTable, HoodieWriteConfig config) {
     this.hoodieTable = hoodieTable;
-    this.fileSystemView = hoodieTable.getCompletedFileSystemView();
+    this.fileSystemView = hoodieTable.getHoodieView();
     this.commitTimeline = hoodieTable.getCompletedCommitTimeline();
     this.config = config;
     this.fgIdToPendingCompactionOperations =
-        ((HoodieTableFileSystemView)hoodieTable.getRTFileSystemView()).getFgIdToPendingCompaction().entrySet()
-            .stream().map(entry -> Pair.of(entry.getKey(), entry.getValue().getValue()))
+        ((SyncableFileSystemView)hoodieTable.getRTFileSystemView()).getPendingCompactionOperations()
+            .map(entry -> Pair.of(new HoodieFileGroupId(entry.getValue().getPartitionPath(),
+                    entry.getValue().getFileId()), entry.getValue()))
             .collect(Collectors.toMap(Pair::getKey, Pair::getValue));
   }
 
@@ -86,7 +86,14 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
 
     for (HoodieFileGroup fileGroup : fileGroups) {
       int keepVersions = config.getCleanerFileVersionsRetained();
-      Iterator<FileSlice> fileSliceIterator = fileGroup.getAllFileSlices().iterator();
+      // do not cleanup slice required for pending compaction
+      Iterator<FileSlice> fileSliceIterator = fileGroup.getAllFileSlices()
+          .filter(fs -> !isFileSliceNeededForPendingCompaction(fs)).iterator();
+      if (isFileGroupInPendingCompaction(fileGroup)) {
+        // We have already saved the last version of file-groups for pending compaction Id
+        keepVersions--;
+      }
+
       while (fileSliceIterator.hasNext() && keepVersions > 0) {
         // Skip this most recent version
         FileSlice nextSlice = fileSliceIterator.next();
@@ -100,16 +107,14 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
       // Delete the remaining files
       while (fileSliceIterator.hasNext()) {
         FileSlice nextSlice = fileSliceIterator.next();
-        if (!isFileSliceNeededForPendingCompaction(nextSlice)) {
-          if (nextSlice.getDataFile().isPresent()) {
-            HoodieDataFile dataFile = nextSlice.getDataFile().get();
-            deletePaths.add(dataFile.getFileStatus().getPath().toString());
-          }
-          if (hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ) {
-            // If merge on read, then clean the log files for the commits as well
-            deletePaths.addAll(nextSlice.getLogFiles().map(file -> file.getPath().toString())
-                .collect(Collectors.toList()));
-          }
+        if (nextSlice.getDataFile().isPresent()) {
+          HoodieDataFile dataFile = nextSlice.getDataFile().get();
+          deletePaths.add(dataFile.getPath());
+        }
+        if (hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ) {
+          // If merge on read, then clean the log files for the commits as well
+          deletePaths.addAll(nextSlice.getLogFiles().map(file -> file.getPath().toString())
+              .collect(Collectors.toList()));
         }
       }
     }
@@ -180,7 +185,7 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
               .compareTimestamps(earliestCommitToRetain.getTimestamp(), fileCommitTime,
                   HoodieTimeline.GREATER)) {
             // this is a commit, that should be cleaned.
-            aFile.ifPresent(hoodieDataFile -> deletePaths.add(hoodieDataFile.getFileStatus().getPath().toString()));
+            aFile.ifPresent(hoodieDataFile -> deletePaths.add(hoodieDataFile.getPath()));
             if (hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ) {
               // If merge on read, then clean the log files for the commits as well
               deletePaths.addAll(aSlice.getLogFiles().map(file -> file.getPath().toString())
@@ -258,4 +263,8 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
     }
     return false;
   }
+
+  private boolean isFileGroupInPendingCompaction(HoodieFileGroup fg) {
+    return fgIdToPendingCompactionOperations.containsKey(fg.getFileGroupId());
+  }
 }
diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java
index 0a22865..734b7c0 100644
--- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java
+++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java
@@ -155,6 +155,7 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieIOH
     logger.info("Closing the file " + writeStatus.getFileId() + " as we are done with all the records "
         + recordsWritten);
     try {
+
       storageWriter.close();
 
       HoodieWriteStat stat = new HoodieWriteStat();
@@ -165,7 +166,9 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieIOH
       stat.setPrevCommit(HoodieWriteStat.NULL_COMMIT);
       stat.setFileId(writeStatus.getFileId());
       stat.setPaths(new Path(config.getBasePath()), path, tempPath);
-      stat.setTotalWriteBytes(FSUtils.getFileSize(fs, getStorageWriterPath()));
+      long fileSizeInBytes = FSUtils.getFileSize(fs, getStorageWriterPath());
+      stat.setTotalWriteBytes(fileSizeInBytes);
+      stat.setFileSizeInBytes(fileSizeInBytes);
       stat.setTotalWriteErrors(writeStatus.getTotalErrorRecords());
       RuntimeStats runtimeStats = new RuntimeStats();
       runtimeStats.setTotalCreateTime(timer.endTimer());
@@ -182,4 +185,4 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieIOH
     // Use tempPath for storage writer if possible
     return (this.tempPath == null) ? this.path : this.tempPath;
   }
-}
\ No newline at end of file
+}
diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java
index a1f8f63..abcd247 100644
--- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java
+++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieIOHandle.java
@@ -20,7 +20,6 @@ import com.uber.hoodie.WriteStatus;
 import com.uber.hoodie.common.model.HoodieRecord;
 import com.uber.hoodie.common.model.HoodieRecordPayload;
 import com.uber.hoodie.common.table.HoodieTableMetaClient;
-import com.uber.hoodie.common.table.HoodieTimeline;
 import com.uber.hoodie.common.util.FSUtils;
 import com.uber.hoodie.common.util.HoodieAvroUtils;
 import com.uber.hoodie.common.util.HoodieTimer;
@@ -48,7 +47,6 @@ public abstract class HoodieIOHandle<T extends HoodieRecordPayload> {
   protected final HoodieTable<T> hoodieTable;
   protected final Schema originalSchema;
   protected final Schema writerSchema;
-  protected HoodieTimeline hoodieTimeline;
   protected HoodieTimer timer;
   protected final WriteStatus writeStatus;
 
@@ -57,7 +55,6 @@ public abstract class HoodieIOHandle<T extends HoodieRecordPayload> {
     this.config = config;
     this.fs = hoodieTable.getMetaClient().getFs();
     this.hoodieTable = hoodieTable;
-    this.hoodieTimeline = hoodieTable.getCompletedCommitsTimeline();
     this.originalSchema = new Schema.Parser().parse(config.getSchema());
     this.writerSchema = createHoodieWriteSchema(originalSchema);
     this.timer = new HoodieTimer().startTimer();
@@ -159,4 +156,4 @@ public abstract class HoodieIOHandle<T extends HoodieRecordPayload> {
   public abstract WriteStatus close();
 
   public abstract WriteStatus getWriteStatus();
-}
\ No newline at end of file
+}
diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java
index 4c3bfa8..ae98eb5 100644
--- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java
+++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java
@@ -24,7 +24,6 @@ import com.uber.hoodie.common.model.HoodieRecordLocation;
 import com.uber.hoodie.common.model.HoodieRecordPayload;
 import com.uber.hoodie.common.model.HoodieWriteStat;
 import com.uber.hoodie.common.model.HoodieWriteStat.RuntimeStats;
-import com.uber.hoodie.common.table.TableFileSystemView;
 import com.uber.hoodie.common.util.DefaultSizeEstimator;
 import com.uber.hoodie.common.util.FSUtils;
 import com.uber.hoodie.common.util.HoodieRecordSizeEstimator;
@@ -56,7 +55,6 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
   private Map<String, HoodieRecord<T>> keyToNewRecords;
   private Set<String> writtenRecordKeys;
   private HoodieStorageWriter<IndexedRecord> storageWriter;
-  private TableFileSystemView.ReadOptimizedView fileSystemView;
   private Path newFilePath;
   private Path oldFilePath;
   private Path tempPath = null;
@@ -69,20 +67,17 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
   public HoodieMergeHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable,
       Iterator<HoodieRecord<T>> recordItr, String fileId) {
     super(config, commitTime, hoodieTable);
-    this.fileSystemView = hoodieTable.getROFileSystemView();
     String partitionPath = init(fileId, recordItr);
     init(fileId, partitionPath,
-        fileSystemView.getLatestDataFiles(partitionPath)
-            .filter(dataFile -> dataFile.getFileId().equals(fileId)).findFirst());
+        hoodieTable.getROFileSystemView().getLatestDataFile(partitionPath, fileId).get());
   }
 
   /**
    * Called by compactor code path
    */
   public HoodieMergeHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable,
-      Map<String, HoodieRecord<T>> keyToNewRecords, String fileId, Optional<HoodieDataFile> dataFileToBeMerged) {
+      Map<String, HoodieRecord<T>> keyToNewRecords, String fileId, HoodieDataFile dataFileToBeMerged) {
     super(config, commitTime, hoodieTable);
-    this.fileSystemView = hoodieTable.getROFileSystemView();
     this.keyToNewRecords = keyToNewRecords;
     this.useWriterSchema = true;
     init(fileId, keyToNewRecords.get(keyToNewRecords.keySet().stream().findFirst().get())
@@ -92,12 +87,11 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
   /**
    * Extract old file path, initialize StorageWriter and WriteStatus
    */
-  private void init(String fileId, String partitionPath, Optional<HoodieDataFile> dataFileToBeMerged) {
+  private void init(String fileId, String partitionPath, HoodieDataFile dataFileToBeMerged) {
     this.writtenRecordKeys = new HashSet<>();
     writeStatus.setStat(new HoodieWriteStat());
     try {
-      //TODO: dataFileToBeMerged must be optional. Will be fixed by Nishith's changes to support insert to log-files
-      String latestValidFilePath = dataFileToBeMerged.get().getFileName();
+      String latestValidFilePath = dataFileToBeMerged.getFileName();
       writeStatus.getStat().setPrevCommit(FSUtils.getCommitTime(latestValidFilePath));
 
       HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs, commitTime,
@@ -276,7 +270,9 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
         storageWriter.close();
       }
 
-      writeStatus.getStat().setTotalWriteBytes(FSUtils.getFileSize(fs, getStorageWriterPath()));
+      long fileSizeInBytes = FSUtils.getFileSize(fs, getStorageWriterPath());
+      writeStatus.getStat().setTotalWriteBytes(fileSizeInBytes);
+      writeStatus.getStat().setFileSizeInBytes(fileSizeInBytes);
       writeStatus.getStat().setNumWrites(recordsWritten);
       writeStatus.getStat().setNumDeletes(recordsDeleted);
       writeStatus.getStat().setNumUpdateWrites(updatedRecordsWritten);
diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java
index b4abbca..86aff63 100644
--- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java
+++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java
@@ -37,6 +37,7 @@ import com.uber.hoodie.common.table.log.HoodieMergedLogRecordScanner;
 import com.uber.hoodie.common.util.CompactionUtils;
 import com.uber.hoodie.common.util.FSUtils;
 import com.uber.hoodie.common.util.HoodieAvroUtils;
+import com.uber.hoodie.common.util.Option;
 import com.uber.hoodie.common.util.collection.Pair;
 import com.uber.hoodie.config.HoodieWriteConfig;
 import com.uber.hoodie.io.compact.strategy.CompactionStrategy;
@@ -89,6 +90,7 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
     List<CompactionOperation> operations = compactionPlan.getOperations().stream().map(
             CompactionOperation::convertFromAvroRecordInstance).collect(toList());
     log.info("Compactor compacting " + operations + " files");
+
     return jsc.parallelize(operations, operations.size())
         .map(s -> compact(table, metaClient, config, s, compactionInstantTime))
         .flatMap(List::iterator);
@@ -113,7 +115,6 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
         .getTimelineOfActions(
             Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.ROLLBACK_ACTION,
                 HoodieTimeline.DELTA_COMMIT_ACTION))
-
         .filterCompletedInstants().lastInstant().get().getTimestamp();
     log.info("MaxMemoryPerCompaction => " + config.getMaxMemoryPerCompaction());
     HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs,
@@ -125,9 +126,8 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
       return Lists.<WriteStatus>newArrayList();
     }
 
-    Optional<HoodieDataFile> oldDataFileOpt = hoodieCopyOnWriteTable.getROFileSystemView()
-        .getLatestDataFilesOn(operation.getPartitionPath(), operation.getBaseInstantTime())
-        .filter(df -> df.getFileId().equals(operation.getFileId())).findFirst();
+    Option<HoodieDataFile> oldDataFileOpt = hoodieCopyOnWriteTable.getROFileSystemView()
+        .getDataFileOn(operation.getPartitionPath(), operation.getBaseInstantTime(), operation.getFileId());
 
     // Compacting is very similar to applying updates to existing file
     Iterator<List<WriteStatus>> result;
@@ -135,7 +135,7 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
     // new base parquet file.
     if (operation.getDataFilePath().isPresent()) {
       result = hoodieCopyOnWriteTable
-          .handleUpdate(commitTime, operation.getFileId(), scanner.getRecords(), oldDataFileOpt);
+          .handleUpdate(commitTime, operation.getFileId(), scanner.getRecords(), oldDataFileOpt.get());
     } else {
       result = hoodieCopyOnWriteTable
           .handleInsert(commitTime, operation.getPartitionPath(), operation.getFileId(), scanner.iterator());
@@ -189,6 +189,7 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
       // In case no partitions could be picked, return no compaction plan
       return null;
     }
+
     TableFileSystemView.RealtimeView fileSystemView = hoodieTable.getRTFileSystemView();
     log.info("Compaction looking for files to compact in " + partitionPaths + " partitions");
     List<HoodieCompactionOperation> operations =
diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/CompactionStrategy.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/CompactionStrategy.java
index 3efc532..bcd8576 100644
--- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/CompactionStrategy.java
+++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/CompactionStrategy.java
@@ -59,8 +59,8 @@ public abstract class CompactionStrategy implements Serializable {
     Map<String, Double> metrics = Maps.newHashMap();
     Long defaultMaxParquetFileSize = writeConfig.getParquetMaxFileSize();
     // Total size of all the log files
-    Long totalLogFileSize = logFiles.stream().map(HoodieLogFile::getFileSize).filter(Optional::isPresent)
-        .map(Optional::get).reduce((size1, size2) -> size1 + size2).orElse(0L);
+    Long totalLogFileSize = logFiles.stream().map(HoodieLogFile::getFileSize).filter(size -> size >= 0)
+        .reduce((size1, size2) -> size1 + size2).orElse(0L);
     // Total read will be the base file + all the log files
     Long totalIORead = FSUtils.getSizeInMB((dataFile.isPresent() ? dataFile.get().getFileSize() : 0L)
         + totalLogFileSize);
diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/LogFileSizeBasedCompactionStrategy.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/LogFileSizeBasedCompactionStrategy.java
index 8dd3669..9b10bcf 100644
--- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/LogFileSizeBasedCompactionStrategy.java
+++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/LogFileSizeBasedCompactionStrategy.java
@@ -46,7 +46,7 @@ public class LogFileSizeBasedCompactionStrategy extends BoundedIOCompactionStrat
 
     // Total size of all the log files
     Long totalLogFileSize = logFiles.stream().map(HoodieLogFile::getFileSize)
-        .filter(Optional::isPresent).map(Optional::get).reduce((size1, size2) -> size1 + size2)
+        .filter(size -> size >= 0).reduce((size1, size2) -> size1 + size2)
         .orElse(0L);
     // save the metrics needed during the order
     metrics.put(TOTAL_LOG_FILE_SIZE, totalLogFileSize.doubleValue());
diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java
index a1d9731..e968e93 100644
--- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java
+++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java
@@ -58,7 +58,6 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
-import java.util.Optional;
 import java.util.Set;
 import java.util.stream.Collectors;
 import org.apache.avro.generic.GenericRecord;
@@ -186,9 +185,9 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
   }
 
   public Iterator<List<WriteStatus>> handleUpdate(String commitTime, String fileId,
-      Map<String, HoodieRecord<T>> keyToNewRecords, Optional<HoodieDataFile> dataFileOpt) throws IOException {
+      Map<String, HoodieRecord<T>> keyToNewRecords, HoodieDataFile oldDataFile) throws IOException {
     // these are updates
-    HoodieMergeHandle upsertHandle = getUpdateHandle(commitTime, fileId, keyToNewRecords, dataFileOpt);
+    HoodieMergeHandle upsertHandle = getUpdateHandle(commitTime, fileId, keyToNewRecords, oldDataFile);
     return handleUpdateInternal(upsertHandle, commitTime, fileId);
   }
 
@@ -231,7 +230,7 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
   }
 
   protected HoodieMergeHandle getUpdateHandle(String commitTime, String fileId,
-      Map<String, HoodieRecord<T>> keyToNewRecords, Optional<HoodieDataFile> dataFileToBeMerged) {
+      Map<String, HoodieRecord<T>> keyToNewRecords, HoodieDataFile dataFileToBeMerged) {
     return new HoodieMergeHandle<>(config, commitTime, this, keyToNewRecords, fileId, dataFileToBeMerged);
   }
 
diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java
index 32daa05..402fc8c 100644
--- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java
+++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java
@@ -32,13 +32,13 @@ import com.uber.hoodie.common.model.HoodieRollingStat;
 import com.uber.hoodie.common.model.HoodieRollingStatMetadata;
 import com.uber.hoodie.common.model.HoodieWriteStat;
 import com.uber.hoodie.common.table.HoodieTimeline;
+import com.uber.hoodie.common.table.SyncableFileSystemView;
 import com.uber.hoodie.common.table.log.HoodieLogFormat;
 import com.uber.hoodie.common.table.log.block.HoodieCommandBlock;
 import com.uber.hoodie.common.table.log.block.HoodieCommandBlock.HoodieCommandBlockTypeEnum;
 import com.uber.hoodie.common.table.log.block.HoodieLogBlock.HeaderMetadataType;
 import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
 import com.uber.hoodie.common.table.timeline.HoodieInstant;
-import com.uber.hoodie.common.table.view.HoodieTableFileSystemView;
 import com.uber.hoodie.common.util.FSUtils;
 import com.uber.hoodie.config.HoodieWriteConfig;
 import com.uber.hoodie.exception.HoodieCompactionException;
@@ -55,7 +55,6 @@ import java.io.UncheckedIOException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -155,8 +154,10 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
     HoodieRealtimeTableCompactor compactor = new HoodieRealtimeTableCompactor();
     try {
       return compactor.generateCompactionPlan(jsc, this, config, instantTime,
-          new HashSet<>(((HoodieTableFileSystemView)getRTFileSystemView())
-              .getFgIdToPendingCompaction().keySet()));
+          ((SyncableFileSystemView)getRTFileSystemView()).getPendingCompactionOperations()
+              .map(instantTimeCompactionopPair -> instantTimeCompactionopPair.getValue().getFileGroupId())
+              .collect(Collectors.toSet()));
+
     } catch (IOException e) {
       throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e);
     }
@@ -460,7 +461,8 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
     // TODO (NA) : Make this static part of utility
     @VisibleForTesting
     public long convertLogFilesSizeToExpectedParquetSize(List<HoodieLogFile> hoodieLogFiles) {
-      long totalSizeOfLogFiles = hoodieLogFiles.stream().map(hoodieLogFile -> hoodieLogFile.getFileSize().get())
+      long totalSizeOfLogFiles = hoodieLogFiles.stream().map(hoodieLogFile -> hoodieLogFile.getFileSize())
+          .filter(size -> size > 0)
           .reduce((a, b) -> (a + b)).orElse(0L);
       // Here we assume that if there is no base parquet file, all log files contain only inserts.
       // We can then just get the parquet equivalent size of these log files, compare that with
diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java
index b8d9609..6f7dd30 100644
--- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java
+++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieTable.java
@@ -21,14 +21,17 @@ import com.uber.hoodie.avro.model.HoodieCompactionPlan;
 import com.uber.hoodie.avro.model.HoodieSavepointMetadata;
 import com.uber.hoodie.common.HoodieCleanStat;
 import com.uber.hoodie.common.HoodieRollbackStat;
+import com.uber.hoodie.common.SerializableConfiguration;
 import com.uber.hoodie.common.model.HoodieRecord;
 import com.uber.hoodie.common.model.HoodieRecordPayload;
 import com.uber.hoodie.common.model.HoodieWriteStat;
 import com.uber.hoodie.common.table.HoodieTableMetaClient;
 import com.uber.hoodie.common.table.HoodieTimeline;
+import com.uber.hoodie.common.table.SyncableFileSystemView;
 import com.uber.hoodie.common.table.TableFileSystemView;
 import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
 import com.uber.hoodie.common.table.timeline.HoodieInstant;
+import com.uber.hoodie.common.table.view.FileSystemViewManager;
 import com.uber.hoodie.common.table.view.HoodieTableFileSystemView;
 import com.uber.hoodie.common.util.AvroUtils;
 import com.uber.hoodie.config.HoodieWriteConfig;
@@ -62,12 +65,26 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
   protected final HoodieTableMetaClient metaClient;
   protected final HoodieIndex<T> index;
 
+  private SerializableConfiguration hadoopConfiguration;
+  private transient FileSystemViewManager viewManager;
+
   protected HoodieTable(HoodieWriteConfig config, JavaSparkContext jsc) {
     this.config = config;
+    this.hadoopConfiguration = new SerializableConfiguration(jsc.hadoopConfiguration());
+    this.viewManager = FileSystemViewManager.createViewManager(
+        new SerializableConfiguration(jsc.hadoopConfiguration()), config.getViewStorageConfig());
     this.metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true);
     this.index = HoodieIndex.createIndex(config, jsc);
   }
 
+  private synchronized FileSystemViewManager getViewManager() {
+    if (null == viewManager) {
+      viewManager = FileSystemViewManager.createViewManager(hadoopConfiguration,
+          config.getViewStorageConfig());
+    }
+    return viewManager;
+  }
+
   public static <T extends HoodieRecordPayload> HoodieTable<T> getHoodieTable(
       HoodieTableMetaClient metaClient, HoodieWriteConfig config, JavaSparkContext jsc) {
     switch (metaClient.getTableType()) {
@@ -118,22 +135,21 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
    * Get the read optimized view of the file system for this table
    */
   public TableFileSystemView.ReadOptimizedView getROFileSystemView() {
-    return new HoodieTableFileSystemView(metaClient, getCompletedCommitsTimeline());
+    return getViewManager().getFileSystemView(metaClient.getBasePath());
   }
 
   /**
    * Get the real time view of the file system for this table
    */
   public TableFileSystemView.RealtimeView getRTFileSystemView() {
-    return new HoodieTableFileSystemView(metaClient,
-        metaClient.getCommitsAndCompactionTimeline().filterCompletedAndCompactionInstants());
+    return getViewManager().getFileSystemView(metaClient.getBasePath());
   }
 
   /**
-   * Get the completed (commit + compaction) view of the file system for this table
+   * Get complete view of the file system for this table with ability to force sync
    */
-  public TableFileSystemView getCompletedFileSystemView() {
-    return new HoodieTableFileSystemView(metaClient, metaClient.getCommitsTimeline());
+  public SyncableFileSystemView getHoodieView() {
+    return getViewManager().getFileSystemView(metaClient.getBasePath());
   }
 
   /**
diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestAsyncCompaction.java b/hoodie-client/src/test/java/com/uber/hoodie/TestAsyncCompaction.java
index b8d8614..3993fe2 100644
--- a/hoodie-client/src/test/java/com/uber/hoodie/TestAsyncCompaction.java
+++ b/hoodie-client/src/test/java/com/uber/hoodie/TestAsyncCompaction.java
@@ -66,7 +66,7 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
     return getConfigBuilder(autoCommit).build();
   }
 
-  private HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit) {
+  protected HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit) {
     return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
         .withAutoCommit(autoCommit).withAssumeDatePartitioning(true).withCompactionConfig(
             HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024).withInlineCompaction(false)
@@ -143,7 +143,7 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
   public void testRollbackInflightIngestionWithPendingCompaction() throws Exception {
     // Rollback inflight ingestion when there is pending compaction
     HoodieWriteConfig cfg = getConfig(false);
-    HoodieWriteClient client = new HoodieWriteClient(jsc, cfg, true);
+    HoodieWriteClient client = getHoodieWriteClient(cfg, true);
 
     String firstInstantTime = "001";
     String secondInstantTime = "004";
@@ -195,7 +195,7 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
   public void testInflightCompaction() throws Exception {
     // There is inflight compaction. Subsequent compaction run must work correctly
     HoodieWriteConfig cfg = getConfig(true);
-    HoodieWriteClient client = new HoodieWriteClient(jsc, cfg, true);
+    HoodieWriteClient client = getHoodieWriteClient(cfg, true);
 
     String firstInstantTime = "001";
     String secondInstantTime = "004";
@@ -211,7 +211,7 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
 
     // Schedule and mark compaction instant as inflight
     HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
-    HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
+    HoodieTable hoodieTable = getHoodieTable(metaClient, cfg);
     scheduleCompaction(compactionInstantTime, client, cfg);
     moveCompactionFromRequestedToInflight(compactionInstantTime, client, cfg);
 
@@ -227,7 +227,7 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
   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 = new HoodieWriteClient(jsc, cfg, true);
+    HoodieWriteClient client = getHoodieWriteClient(cfg, true);
 
     String firstInstantTime = "001";
     String secondInstantTime = "004";
@@ -264,7 +264,7 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
     // Case: Failure case. Earliest ingestion inflight instant time must be later than compaction time
 
     HoodieWriteConfig cfg = getConfig(false);
-    HoodieWriteClient client = new HoodieWriteClient(jsc, cfg, true);
+    HoodieWriteClient client = getHoodieWriteClient(cfg, true);
 
     String firstInstantTime = "001";
     String secondInstantTime = "004";
@@ -301,7 +301,7 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
     // Case: Failure case. Earliest ingestion inflight instant time must be later than compaction time
 
     HoodieWriteConfig cfg = getConfig(false);
-    HoodieWriteClient client = new HoodieWriteClient(jsc, cfg, true);
+    HoodieWriteClient client = getHoodieWriteClient(cfg, true);
 
     String firstInstantTime = "001";
     String secondInstantTime = "004";
@@ -351,7 +351,7 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
   public void testCompactionAfterTwoDeltaCommits() throws Exception {
     // No Delta Commits after compaction request
     HoodieWriteConfig cfg = getConfig(true);
-    HoodieWriteClient client = new HoodieWriteClient(jsc, cfg, true);
+    HoodieWriteClient client = getHoodieWriteClient(cfg, true);
 
     String firstInstantTime = "001";
     String secondInstantTime = "004";
@@ -363,7 +363,7 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
         records, cfg, true, new ArrayList<>());
 
     HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
-    HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
+    HoodieTable hoodieTable = getHoodieTable(metaClient, cfg);
     scheduleAndExecuteCompaction(compactionInstantTime, client, hoodieTable, cfg, numRecs, false);
   }
 
@@ -371,7 +371,7 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
   public void testInterleavedCompaction() throws Exception {
     //Case: Two delta commits before and after compaction schedule
     HoodieWriteConfig cfg = getConfig(true);
-    HoodieWriteClient client = new HoodieWriteClient(jsc, cfg, true);
+    HoodieWriteClient client = getHoodieWriteClient(cfg, true);
 
     String firstInstantTime = "001";
     String secondInstantTime = "004";
@@ -386,7 +386,7 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
         records, cfg, true, new ArrayList<>());
 
     HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
-    HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
+    HoodieTable hoodieTable = getHoodieTable(metaClient, cfg);
     scheduleCompaction(compactionInstantTime, client, cfg);
 
     runNextDeltaCommits(client, Arrays.asList(thirdInstantTime, fourthInstantTime),
@@ -402,12 +402,11 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
       final Map<HoodieFileGroupId, Pair<String, HoodieCompactionOperation>> fgIdToCompactionOperation,
       HoodieWriteConfig cfg) throws IOException {
     HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
-    HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
+    HoodieTable table = getHoodieTable(metaClient, cfg);
     List<FileSlice> fileSliceList = getCurrentLatestFileSlices(table, cfg);
     fileSliceList.forEach(fileSlice -> {
       Pair<String, HoodieCompactionOperation> opPair = fgIdToCompactionOperation.get(fileSlice.getFileGroupId());
       if (opPair != null) {
-        System.out.println("FileSlice :" + fileSlice);
         assertTrue("Expect baseInstant to match compaction Instant",
             fileSlice.getBaseInstantTime().equals(opPair.getKey()));
         assertTrue("Expect atleast one log file to be present where the latest delta commit was written",
@@ -448,7 +447,7 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
       }
       TestHoodieClientBase.assertNoWriteErrors(statusList);
       metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
-      HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
+      HoodieTable hoodieTable = getHoodieTable(metaClient, cfg);
       List<HoodieDataFile> dataFilesToRead = getCurrentLatestDataFiles(hoodieTable, cfg);
       assertTrue("RealtimeTableView should list the parquet files we wrote in the delta commit",
           dataFilesToRead.stream().findAny().isPresent());
@@ -515,8 +514,8 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
     }
 
     // verify that there is a commit
-    table = HoodieTable.getHoodieTable(
-        new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath(), true), cfg, jsc);
+    table = getHoodieTable(
+        new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath(), true), cfg);
     HoodieTimeline timeline = table.getMetaClient().getCommitTimeline().filterCompletedInstants();
     String latestCompactionCommitTime = timeline.lastInstant().get().getTimestamp();
     assertEquals("Expect compaction instant time to be the latest commit time",
@@ -571,4 +570,8 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
   protected HoodieTableType getTableType() {
     return HoodieTableType.MERGE_ON_READ;
   }
+
+  protected HoodieTable getHoodieTable(HoodieTableMetaClient metaClient, HoodieWriteConfig config) {
+    return HoodieTable.getHoodieTable(metaClient, config, jsc);
+  }
 }
diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestAsyncCompactionWithEmbeddedTimelineServer.java b/hoodie-client/src/test/java/com/uber/hoodie/TestAsyncCompactionWithEmbeddedTimelineServer.java
new file mode 100644
index 0000000..b791160
--- /dev/null
+++ b/hoodie-client/src/test/java/com/uber/hoodie/TestAsyncCompactionWithEmbeddedTimelineServer.java
@@ -0,0 +1,56 @@
+/*
+ *  Copyright (c) 2017 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  You may obtain a copy of the License at
+ *
+ *           http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ *
+ */
+
+package com.uber.hoodie;
+
+import com.uber.hoodie.common.table.HoodieTableMetaClient;
+import com.uber.hoodie.common.table.SyncableFileSystemView;
+import com.uber.hoodie.common.table.view.FileSystemViewStorageConfig;
+import com.uber.hoodie.common.table.view.FileSystemViewStorageType;
+import com.uber.hoodie.config.HoodieWriteConfig;
+import com.uber.hoodie.exception.HoodieException;
+import com.uber.hoodie.table.HoodieTable;
+import java.io.IOException;
+import org.junit.After;
+
+public class TestAsyncCompactionWithEmbeddedTimelineServer extends TestAsyncCompaction {
+
+  @Override
+  protected HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit) {
+    HoodieWriteConfig.Builder builder = super.getConfigBuilder(autoCommit);
+    try {
+      return builder.withEmbeddedTimelineServerEnabled(true).withFileSystemViewConfig(
+          FileSystemViewStorageConfig.newBuilder().withStorageType(FileSystemViewStorageType.EMBEDDED_KV_STORE)
+              .build());
+    } catch (Exception e) {
+      throw new HoodieException(e);
+    }
+  }
+
+  @Override
+  protected HoodieTable getHoodieTable(HoodieTableMetaClient metaClient, HoodieWriteConfig config) {
+    HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
+    ((SyncableFileSystemView) (table.getRTFileSystemView())).reset();
+    return table;
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    super.tearDown();
+  }
+}
diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java b/hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java
index 98775be..987a1d2 100644
--- a/hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java
+++ b/hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java
@@ -124,7 +124,7 @@ public class TestCleaner extends TestHoodieClientBase {
     HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
     assertEquals("Expecting a single commit.", 1, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants());
     // Should have 100 records in table (check using Index), all in locations marked at commit
-    HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig(), jsc);
+    HoodieTable table = HoodieTable.getHoodieTable(metaClient, client.config, jsc);
 
     assertFalse(table.getCompletedCommitsTimeline().empty());
     String commitTime = table.getCompletedCommitsTimeline().getInstants().findFirst().get().getTimestamp();
@@ -194,7 +194,7 @@ public class TestCleaner extends TestHoodieClientBase {
         .withParallelism(1, 1).withBulkInsertParallelism(1)
         .withFinalizeWriteParallelism(1).withConsistencyCheckEnabled(true)
         .build();
-    HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
+    HoodieWriteClient client = getHoodieWriteClient(cfg);
 
     final Function2<List<HoodieRecord>, String, Integer> recordInsertGenWrappedFunction =
         generateWrapRecordsFn(isPreppedAPI, cfg, dataGen::generateInserts);
@@ -355,7 +355,7 @@ public class TestCleaner extends TestHoodieClientBase {
             .withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainCommits(maxCommits).build())
         .withParallelism(1, 1).withBulkInsertParallelism(1)
         .withFinalizeWriteParallelism(1).withConsistencyCheckEnabled(true).build();
-    HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
+    HoodieWriteClient client = getHoodieWriteClient(cfg);
 
     final Function2<List<HoodieRecord>, String, Integer> recordInsertGenWrappedFunction =
         generateWrapRecordsFn(isPreppedAPI, cfg, dataGen::generateInserts);
diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestCleanerWithEmbeddedTimelineServer.java b/hoodie-client/src/test/java/com/uber/hoodie/TestCleanerWithEmbeddedTimelineServer.java
new file mode 100644
index 0000000..8444a45
--- /dev/null
+++ b/hoodie-client/src/test/java/com/uber/hoodie/TestCleanerWithEmbeddedTimelineServer.java
@@ -0,0 +1,45 @@
+/*
+ *  Copyright (c) 2019 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  You may obtain a copy of the License at
+ *
+ *           http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package com.uber.hoodie;
+
+import com.uber.hoodie.common.table.view.FileSystemViewStorageConfig;
+import com.uber.hoodie.common.table.view.FileSystemViewStorageType;
+import com.uber.hoodie.config.HoodieWriteConfig;
+import com.uber.hoodie.exception.HoodieException;
+import java.io.IOException;
+import org.junit.After;
+
+public class TestCleanerWithEmbeddedTimelineServer extends TestCleaner {
+
+  @Override
+  HoodieWriteConfig.Builder getConfigBuilder() {
+    HoodieWriteConfig.Builder builder = super.getConfigBuilder();
+    try {
+      builder.withEmbeddedTimelineServerEnabled(true).withFileSystemViewConfig(
+          FileSystemViewStorageConfig.newBuilder().withStorageType(FileSystemViewStorageType.EMBEDDED_KV_STORE)
+              .build());
+    } catch (Exception e) {
+      throw new HoodieException(e);
+    }
+    return builder;
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    super.tearDown();
+  }
+}
diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestClientRollback.java b/hoodie-client/src/test/java/com/uber/hoodie/TestClientRollback.java
index 1753186..ad8e5eb 100644
--- a/hoodie-client/src/test/java/com/uber/hoodie/TestClientRollback.java
+++ b/hoodie-client/src/test/java/com/uber/hoodie/TestClientRollback.java
@@ -61,7 +61,7 @@ public class TestClientRollback extends TestHoodieClientBase {
     HoodieWriteConfig cfg = getConfigBuilder().withCompactionConfig(
         HoodieCompactionConfig.newBuilder().withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(1)
             .build()).build();
-    HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
+    HoodieWriteClient client = getHoodieWriteClient(cfg);
     HoodieTestDataGenerator.writePartitionMetadata(fs, HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, basePath);
 
     /**
diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientBase.java b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientBase.java
index 2df095e..a6f529f 100644
--- a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientBase.java
+++ b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientBase.java
@@ -75,6 +75,28 @@ public class TestHoodieClientBase implements Serializable {
   protected TemporaryFolder folder = null;
   protected transient HoodieTestDataGenerator dataGen = null;
 
+  private HoodieWriteClient writeClient;
+
+  protected HoodieWriteClient getHoodieWriteClient(HoodieWriteConfig cfg) throws Exception {
+    closeClient();
+    writeClient = new HoodieWriteClient(jsc, cfg);
+    return writeClient;
+  }
+
+  protected HoodieWriteClient getHoodieWriteClient(HoodieWriteConfig cfg, boolean rollbackInflightCommit)
+      throws Exception {
+    closeClient();
+    writeClient = new HoodieWriteClient(jsc, cfg, rollbackInflightCommit);
+    return writeClient;
+  }
+
+  private void closeClient() {
+    if (null != writeClient) {
+      writeClient.close();
+      writeClient = null;
+    }
+  }
+
   @Before
   public void init() throws IOException {
     // Initialize a local spark env
@@ -105,6 +127,8 @@ public class TestHoodieClientBase implements Serializable {
    * Properly release resources at end of each test
    */
   public void tearDown() throws IOException {
+    closeClient();
+
     if (null != sqlContext) {
       logger.info("Clearing sql context cache of spark-session used in previous test-case");
       sqlContext.clearCache();
diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java
index 427b6b0..9ce1070 100644
--- a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java
+++ b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java
@@ -133,7 +133,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
       boolean isPrepped) throws Exception {
     // Set autoCommit false
     HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build();
-    HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
+    HoodieWriteClient client = getHoodieWriteClient(cfg);
 
     String prevCommitTime = "000";
     String newCommitTime = "001";
@@ -428,7 +428,8 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
 
     assertEquals("2 files needs to be committed.", 2, statuses.size());
     HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
-    HoodieTable table = HoodieTable.getHoodieTable(metadata, config, jsc);
+
+    HoodieTable table = getHoodieTable(metadata, config);
     TableFileSystemView.ReadOptimizedView fileSystemView = table.getROFileSystemView();
     List<HoodieDataFile> files = fileSystemView.getLatestDataFilesBeforeOrOn(testPartitionPath, commitTime3)
         .collect(Collectors.toList());
@@ -467,6 +468,10 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
     assertEquals("Total inserts in commit3 must add up", keys3.size(), numTotalInsertsInCommit3);
   }
 
+  protected HoodieTable getHoodieTable(HoodieTableMetaClient metaClient, HoodieWriteConfig config) {
+    return HoodieTable.getHoodieTable(metaClient, config, jsc);
+  }
+
   /**
    * Test scenario of new file-group getting added during insert()
    */
@@ -532,7 +537,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
     assertEquals("2 files needs to be committed.", 2, statuses.size());
 
     HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
-    HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
+    HoodieTable table = getHoodieTable(metaClient, config);
     List<HoodieDataFile> files = table.getROFileSystemView()
         .getLatestDataFilesBeforeOrOn(testPartitionPath, commitTime3)
         .collect(Collectors.toList());
@@ -555,7 +560,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
   public void testCommitWritesRelativePaths() throws Exception {
 
     HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build();
-    HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
+    HoodieWriteClient client = getHoodieWriteClient(cfg);
     HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
     HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
 
@@ -602,7 +607,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
   public void testRollingStatsInMetadata() throws Exception {
 
     HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build();
-    HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
+    HoodieWriteClient client = getHoodieWriteClient(cfg);
     HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
     HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
 
@@ -674,7 +679,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
   @Test
   public void testConsistencyCheckDuringFinalize() throws Exception {
     HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build();
-    HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
+    HoodieWriteClient client = getHoodieWriteClient(cfg);
     HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
         basePath);
 
diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieReadClientWithEmbeddedServer.java b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieReadClientWithEmbeddedServer.java
new file mode 100644
index 0000000..8ecf30e
--- /dev/null
+++ b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieReadClientWithEmbeddedServer.java
@@ -0,0 +1,45 @@
+/*
+ * Copyright (c) 2019 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *          http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.uber.hoodie;
+
+import com.uber.hoodie.common.table.view.FileSystemViewStorageConfig;
+import com.uber.hoodie.common.table.view.FileSystemViewStorageType;
+import com.uber.hoodie.config.HoodieWriteConfig;
+import com.uber.hoodie.exception.HoodieException;
+import java.io.IOException;
+import org.junit.After;
+
+public class TestHoodieReadClientWithEmbeddedServer extends TestHoodieReadClient {
+
+  @Override
+  HoodieWriteConfig.Builder getConfigBuilder() {
+    HoodieWriteConfig.Builder builder = super.getConfigBuilder();
+    try {
+      builder.withEmbeddedTimelineServerEnabled(true).withFileSystemViewConfig(
+          FileSystemViewStorageConfig.newBuilder().withStorageType(FileSystemViewStorageType.EMBEDDED_KV_STORE)
+              .build());
+    } catch (Exception e) {
+      throw new HoodieException(e);
+    }
+    return builder;
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    super.tearDown();
+  }
+}
diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieWriteClientWithEmbeddedServer.java b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieWriteClientWithEmbeddedServer.java
new file mode 100644
index 0000000..b982ffd
--- /dev/null
+++ b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieWriteClientWithEmbeddedServer.java
@@ -0,0 +1,55 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *          http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.uber.hoodie;
+
+import com.uber.hoodie.common.table.HoodieTableMetaClient;
+import com.uber.hoodie.common.table.SyncableFileSystemView;
+import com.uber.hoodie.common.table.view.FileSystemViewStorageConfig;
+import com.uber.hoodie.common.table.view.FileSystemViewStorageType;
+import com.uber.hoodie.config.HoodieWriteConfig;
+import com.uber.hoodie.exception.HoodieException;
+import com.uber.hoodie.table.HoodieTable;
+import java.io.IOException;
+import org.junit.After;
+
+public class TestHoodieWriteClientWithEmbeddedServer extends TestHoodieClientOnCopyOnWriteStorage {
+
+  @Override
+  HoodieWriteConfig.Builder getConfigBuilder() {
+    HoodieWriteConfig.Builder builder = super.getConfigBuilder();
+    try {
+      builder.withEmbeddedTimelineServerEnabled(true).withFileSystemViewConfig(
+          FileSystemViewStorageConfig.newBuilder().withStorageType(FileSystemViewStorageType.EMBEDDED_KV_STORE)
+              .build());
+    } catch (Exception e) {
+      throw new HoodieException(e);
+    }
+    return builder;
+  }
+
+  @Override
+  protected HoodieTable getHoodieTable(HoodieTableMetaClient metaClient, HoodieWriteConfig config) {
+    HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
+    ((SyncableFileSystemView) (table.getRTFileSystemView())).reset();
+    return table;
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    super.tearDown();
+  }
+}
diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestMultiFS.java b/hoodie-client/src/test/java/com/uber/hoodie/TestMultiFS.java
index c9969f2..d69c469 100644
--- a/hoodie-client/src/test/java/com/uber/hoodie/TestMultiFS.java
+++ b/hoodie-client/src/test/java/com/uber/hoodie/TestMultiFS.java
@@ -60,7 +60,7 @@ public class TestMultiFS implements Serializable {
   private static JavaSparkContext jsc;
   private static SQLContext sqlContext;
   private String tablePath = "file:///tmp/hoodie/sample-table";
-  private String tableName = "hoodie_rt";
+  protected String tableName = "hoodie_rt";
   private String tableType = HoodieTableType.COPY_ON_WRITE.name();
 
   @BeforeClass
@@ -96,6 +96,13 @@ public class TestMultiFS implements Serializable {
     FileSystem.closeAll();
   }
 
+  protected HoodieWriteConfig getHoodieWriteConfig(String basePath) {
+    return HoodieWriteConfig.newBuilder().withPath(basePath)
+        .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
+        .forTable(tableName).withIndexConfig(
+        HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()).build();
+  }
+
   @Test
   public void readLocalWriteHDFS() throws Exception {
 
@@ -108,10 +115,7 @@ public class TestMultiFS implements Serializable {
             HoodieAvroPayload.class.getName());
 
     //Create write client to write some records in
-    HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(dfsBasePath)
-        .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
-        .forTable(tableName).withIndexConfig(
-            HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()).build();
+    HoodieWriteConfig cfg = getHoodieWriteConfig(dfsBasePath);
     HoodieWriteClient hdfsWriteClient = new HoodieWriteClient(jsc, cfg);
 
     // Write generated data to hdfs (only inserts)
@@ -132,10 +136,7 @@ public class TestMultiFS implements Serializable {
     HoodieTableMetaClient
         .initTableType(jsc.hadoopConfiguration(), tablePath, HoodieTableType.valueOf(tableType), tableName,
             HoodieAvroPayload.class.getName());
-    HoodieWriteConfig localConfig = HoodieWriteConfig.newBuilder().withPath(tablePath)
-        .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
-        .forTable(tableName).withIndexConfig(
-            HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()).build();
+    HoodieWriteConfig localConfig = getHoodieWriteConfig(tablePath);
     HoodieWriteClient localWriteClient = new HoodieWriteClient(jsc, localConfig);
 
     String writeCommitTime = localWriteClient.startCommit();
@@ -151,5 +152,8 @@ public class TestMultiFS implements Serializable {
     timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
     Dataset<Row> localReadRecords = HoodieClientTestUtils.readCommit(tablePath, sqlContext, timeline, writeCommitTime);
     assertEquals("Should contain 100 records", localReadRecords.count(), localRecords.size());
+
+    hdfsWriteClient.close();
+    localWriteClient.close();
   }
 }
diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestMultiFSWithEmbeddedServer.java b/hoodie-client/src/test/java/com/uber/hoodie/TestMultiFSWithEmbeddedServer.java
new file mode 100644
index 0000000..1c14918
--- /dev/null
+++ b/hoodie-client/src/test/java/com/uber/hoodie/TestMultiFSWithEmbeddedServer.java
@@ -0,0 +1,36 @@
+/*
+ * Copyright (c) 2019 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *          http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.uber.hoodie;
+
+import com.uber.hoodie.common.HoodieTestDataGenerator;
+import com.uber.hoodie.config.HoodieIndexConfig;
+import com.uber.hoodie.config.HoodieWriteConfig;
+import com.uber.hoodie.index.HoodieIndex;
+
+/**
+ * Tests MultiFS with embedded timeline server enabled
+ */
+public class TestMultiFSWithEmbeddedServer extends TestMultiFS {
+
+  @Override
+  protected HoodieWriteConfig getHoodieWriteConfig(String basePath) {
+    return HoodieWriteConfig.newBuilder().withPath(basePath).withEmbeddedTimelineServerEnabled(true)
+        .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
+        .forTable(tableName).withIndexConfig(
+            HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()).build();
+  }
+}
diff --git a/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieDataFile.java b/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieDataFile.java
index 88058b3..4d1206c 100644
--- a/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieDataFile.java
+++ b/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieDataFile.java
@@ -24,7 +24,7 @@ public class TestHoodieDataFile extends HoodieDataFile {
   private final long size;
 
   public TestHoodieDataFile(long size) {
-    super(null);
+    super("/tmp/XYXYXYXYXYYX_11_20180918020003.parquet");
     this.size = size;
   }
 
diff --git a/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieLogFile.java b/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieLogFile.java
index df46ec2..d3912d0 100644
--- a/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieLogFile.java
+++ b/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieLogFile.java
@@ -17,7 +17,6 @@
 package com.uber.hoodie.io.strategy;
 
 import com.uber.hoodie.common.model.HoodieLogFile;
-import java.util.Optional;
 import org.apache.hadoop.fs.Path;
 
 public class TestHoodieLogFile extends HoodieLogFile {
@@ -25,7 +24,7 @@ public class TestHoodieLogFile extends HoodieLogFile {
   private final long size;
 
   public TestHoodieLogFile(long size) {
-    super((Path) null);
+    super("/tmp/.ce481ee7-9e53-4a2e-9992-f9e295fa79c0_20180919184844.log.1");
     this.size = size;
   }
 
@@ -39,7 +38,7 @@ public class TestHoodieLogFile extends HoodieLogFile {
   }
 
   @Override
-  public Optional<Long> getFileSize() {
-    return Optional.of(size);
+  public long getFileSize() {
+    return size;
   }
 }
diff --git a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java
index 3687214..3abbe6c 100644
--- a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java
+++ b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java
@@ -23,6 +23,7 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
+import com.uber.hoodie.HoodieReadClient;
 import com.uber.hoodie.HoodieWriteClient;
 import com.uber.hoodie.WriteStatus;
 import com.uber.hoodie.common.HoodieClientTestUtils;
@@ -42,6 +43,7 @@ import com.uber.hoodie.common.model.HoodieTableType;
 import com.uber.hoodie.common.model.HoodieTestUtils;
 import com.uber.hoodie.common.table.HoodieTableMetaClient;
 import com.uber.hoodie.common.table.HoodieTimeline;
+import com.uber.hoodie.common.table.SyncableFileSystemView;
 import com.uber.hoodie.common.table.TableFileSystemView;
 import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
 import com.uber.hoodie.common.table.timeline.HoodieInstant;
@@ -53,7 +55,6 @@ import com.uber.hoodie.config.HoodieStorageConfig;
 import com.uber.hoodie.config.HoodieWriteConfig;
 import com.uber.hoodie.index.HoodieIndex;
 import com.uber.hoodie.index.HoodieIndex.IndexType;
-import com.uber.hoodie.index.bloom.HoodieBloomIndex;
 import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
@@ -83,7 +84,7 @@ import org.junit.rules.TemporaryFolder;
 
 public class TestMergeOnReadTable {
 
-  private static String basePath = null;
+  protected String basePath = null;
   //NOTE : Be careful in using DFS (FileSystem.class) vs LocalFs(RawLocalFileSystem.class)
   //The implementation and gurantees of many API's differ, for example check rename(src,dst)
   private static MiniDFSCluster dfsCluster;
@@ -97,6 +98,9 @@ public class TestMergeOnReadTable {
     if (hdfsTestService != null) {
       hdfsTestService.stop();
       dfsCluster.shutdown();
+      dfsCluster = null;
+      dfs = null;
+      hdfsTestService = null;
     }
     // Need to closeAll to clear FileSystem.Cache, required because DFS and LocalFS used in the
     // same JVM
@@ -441,14 +445,7 @@ public class TestMergeOnReadTable {
      */
     final String commitTime1 = "002";
     // WriteClient with custom config (disable small file handling)
-    client = new HoodieWriteClient(jsc, HoodieWriteConfig.newBuilder().withPath(basePath)
-        .withSchema(TRIP_EXAMPLE_SCHEMA)
-        .withParallelism(2, 2)
-        .withAutoCommit(false).withAssumeDatePartitioning(true).withCompactionConfig(HoodieCompactionConfig.newBuilder()
-            .compactionSmallFileSize(1 * 1024).withInlineCompaction(false)
-            .withMaxNumDeltaCommitsBeforeCompaction(1).build())
-        .withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1 * 1024).build())
-        .forTable("test-trip-table").build());
+    client = new HoodieWriteClient(jsc, getHoodieWriteConfigWithSmallFileHandlingOff());
     client.startCommitWithTime(commitTime1);
 
     List<HoodieRecord> copyOfRecords = new ArrayList<>(records);
@@ -614,14 +611,7 @@ public class TestMergeOnReadTable {
     newCommitTime = "002";
     allCommits.add(newCommitTime);
     // WriteClient with custom config (disable small file handling)
-    HoodieWriteClient nClient = new HoodieWriteClient(jsc, HoodieWriteConfig.newBuilder().withPath(basePath)
-        .withSchema(TRIP_EXAMPLE_SCHEMA)
-        .withParallelism(2, 2)
-        .withAutoCommit(false).withAssumeDatePartitioning(true).withCompactionConfig(HoodieCompactionConfig.newBuilder()
-            .compactionSmallFileSize(1 * 1024).withInlineCompaction(false)
-            .withMaxNumDeltaCommitsBeforeCompaction(1).build())
-        .withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1 * 1024).build())
-        .forTable("test-trip-table").build());
+    HoodieWriteClient nClient = new HoodieWriteClient(jsc, getHoodieWriteConfigWithSmallFileHandlingOff());
     nClient.startCommitWithTime(newCommitTime);
 
     List<HoodieRecord> copyOfRecords = new ArrayList<>(records);
@@ -736,6 +726,16 @@ public class TestMergeOnReadTable {
     assertTrue(fileGroups.isEmpty());
   }
 
+  protected HoodieWriteConfig getHoodieWriteConfigWithSmallFileHandlingOff() {
+    return HoodieWriteConfig.newBuilder().withPath(basePath)
+        .withSchema(TRIP_EXAMPLE_SCHEMA)
+        .withParallelism(2, 2)
+        .withAutoCommit(false).withAssumeDatePartitioning(true).withCompactionConfig(HoodieCompactionConfig.newBuilder()
+            .compactionSmallFileSize(1 * 1024).withInlineCompaction(false)
+            .withMaxNumDeltaCommitsBeforeCompaction(1).build())
+        .withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1 * 1024).build())
+        .forTable("test-trip-table").build();
+  }
 
   @Test
   public void testUpsertPartitioner() throws Exception {
@@ -834,13 +834,14 @@ public class TestMergeOnReadTable {
     HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
     HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
 
+    HoodieTimeline timeline2 = metaClient.getActiveTimeline();
     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();
+    HoodieReadClient readClient = new HoodieReadClient(jsc, config);
+    updatedRecords = readClient.tagLocation(updatedRecordsRDD).collect();
 
     // Write them to corresponding avro logfiles
     HoodieTestUtils
@@ -850,6 +851,9 @@ public class TestMergeOnReadTable {
     // Verify that all data file has one log file
     metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
     table = HoodieTable.getHoodieTable(metaClient, config, jsc);
+    // In writeRecordsToLogFiles, no commit files are getting added, so resetting file-system view state
+    ((SyncableFileSystemView)(table.getRTFileSystemView())).reset();
+
     for (String partitionPath : dataGen.getPartitionPaths()) {
       List<FileSlice> groupedLogFiles = table.getRTFileSystemView().getLatestFileSlices(partitionPath)
           .collect(Collectors.toList());
@@ -1065,6 +1069,9 @@ public class TestMergeOnReadTable {
     writeClient.commitCompaction(newCommitTime, statuses, Optional.empty());
     // Trigger a rollback of compaction
     writeClient.rollback(newCommitTime);
+    table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath), config, jsc);
+    tableRTFileSystemView = table.getRTFileSystemView();
+    ((SyncableFileSystemView)tableRTFileSystemView).reset();
     for (String partitionPath : dataGen.getPartitionPaths()) {
       Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath).filter(fileSlice ->
           fileSlice.getDataFile().isPresent()).count() == 0);
@@ -1292,13 +1299,14 @@ public class TestMergeOnReadTable {
     return getConfigBuilder(autoCommit).build();
   }
 
-  private HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit) {
+  protected HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit) {
     return getConfigBuilder(autoCommit, IndexType.BLOOM);
   }
 
-  private HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit, HoodieIndex.IndexType indexType) {
+  protected HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit, HoodieIndex.IndexType indexType) {
     return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
-        .withAutoCommit(autoCommit).withAssumeDatePartitioning(true).withCompactionConfig(
+        .withAutoCommit(autoCommit).withAssumeDatePartitioning(true)
+        .withCompactionConfig(
             HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024).withInlineCompaction(false)
                 .withMaxNumDeltaCommitsBeforeCompaction(1).build())
         .withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024 * 1024).build())
diff --git a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeonReadTableWithEmbeddedServer.java b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeonReadTableWithEmbeddedServer.java
new file mode 100644
index 0000000..44e57b6
--- /dev/null
+++ b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeonReadTableWithEmbeddedServer.java
@@ -0,0 +1,56 @@
+/*
+ *  Copyright (c) 2019 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  You may obtain a copy of the License at
+ *
+ *           http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ *
+ */
+
+package com.uber.hoodie.table;
+
+import static com.uber.hoodie.common.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA;
+
+import com.uber.hoodie.config.HoodieCompactionConfig;
+import com.uber.hoodie.config.HoodieIndexConfig;
+import com.uber.hoodie.config.HoodieStorageConfig;
+import com.uber.hoodie.config.HoodieWriteConfig;
+import com.uber.hoodie.index.HoodieIndex;
+
+public class TestMergeonReadTableWithEmbeddedServer extends TestMergeOnReadTable {
+
+  @Override
+  protected HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit, HoodieIndex.IndexType indexType) {
+    return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
+        .withAutoCommit(autoCommit).withAssumeDatePartitioning(true)
+        .withCompactionConfig(
+            HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024).withInlineCompaction(false)
+                .withMaxNumDeltaCommitsBeforeCompaction(1).build())
+        .withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024 * 1024).build())
+        .withEmbeddedTimelineServerEnabled(true)
+        .forTable("test-trip-table")
+        .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType).build());
+  }
+
+  @Override
+  protected HoodieWriteConfig getHoodieWriteConfigWithSmallFileHandlingOff() {
+    return HoodieWriteConfig.newBuilder().withPath(basePath)
+        .withSchema(TRIP_EXAMPLE_SCHEMA)
+        .withParallelism(2, 2)
+        .withAutoCommit(false).withAssumeDatePartitioning(true).withCompactionConfig(HoodieCompactionConfig.newBuilder()
+            .compactionSmallFileSize(1 * 1024).withInlineCompaction(false)
+            .withMaxNumDeltaCommitsBeforeCompaction(1).build())
+        .withEmbeddedTimelineServerEnabled(true)
+        .withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1 * 1024).build())
+        .forTable("test-trip-table").build();
+  }
+}
\ No newline at end of file
diff --git a/hoodie-common/pom.xml b/hoodie-common/pom.xml
index 6d3b901..52945a4 100644
--- a/hoodie-common/pom.xml
+++ b/hoodie-common/pom.xml
@@ -71,6 +71,10 @@
 
   <dependencies>
     <dependency>
+      <groupId>org.rocksdb</groupId>
+      <artifactId>rocksdbjni</artifactId>
+    </dependency>
+    <dependency>
       <groupId>org.apache.avro</groupId>
       <artifactId>avro</artifactId>
     </dependency>
@@ -121,9 +125,22 @@
       <classifier>tests</classifier>
     </dependency>
     <dependency>
+      <groupId>org.apache.httpcomponents</groupId>
+      <artifactId>httpclient</artifactId>
+      <version>4.5.4</version>
+    </dependency>
+    <dependency>
+      <groupId>commons-codec</groupId>
+      <artifactId>commons-codec</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.httpcomponents</groupId>
+      <artifactId>fluent-hc</artifactId>
+      <version>4.5.4</version>
+    </dependency>
+    <dependency>
       <groupId>com.esotericsoftware</groupId>
       <artifactId>kryo</artifactId>
-      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.avro</groupId>
diff --git a/hoodie-common/src/main/avro/HoodieCommitMetadata.avsc b/hoodie-common/src/main/avro/HoodieCommitMetadata.avsc
index c46825c..c5e8ce3 100644
--- a/hoodie-common/src/main/avro/HoodieCommitMetadata.avsc
+++ b/hoodie-common/src/main/avro/HoodieCommitMetadata.avsc
@@ -92,6 +92,11 @@
                         "name":"totalRollbackBlocks",
                         "type":["null","long"],
                         "default" : null
+                     },
+                     {
+                        "name":"fileSizeInBytes",
+                        "type":["null","long"],
+                        "default" : null
                      }
                   ]
                }
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/CompactionOperation.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/CompactionOperation.java
index 93a7f2e..e369f71 100644
--- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/CompactionOperation.java
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/CompactionOperation.java
@@ -16,15 +16,17 @@
 
 package com.uber.hoodie.common.model;
 
-import com.google.common.base.Optional;
 import com.uber.hoodie.avro.model.HoodieCompactionOperation;
 import com.uber.hoodie.common.util.FSUtils;
+import com.uber.hoodie.common.util.Option;
 import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 import java.util.stream.Collectors;
+import org.apache.hadoop.fs.Path;
 
 /**
  * Encapsulates all the needed information about a compaction and make a decision whether this
@@ -34,10 +36,9 @@ import java.util.stream.Collectors;
 public class CompactionOperation implements Serializable {
 
   private String baseInstantTime;
-  // Using Guava Optional as it is serializable
-  private Optional<String> dataFileCommitTime;
+  private Option<String> dataFileCommitTime;
   private List<String> deltaFilePaths;
-  private Optional<String> dataFilePath;
+  private Option<String> dataFilePath;
   private HoodieFileGroupId id;
   private Map<String, Double> metrics;
 
@@ -46,19 +47,30 @@ public class CompactionOperation implements Serializable {
   public CompactionOperation() {
   }
 
+  public CompactionOperation(String fileId, String partitionPath, String baseInstantTime,
+      Option<String> dataFileCommitTime, List<String> deltaFilePaths, Option<String> dataFilePath,
+      Map<String, Double> metrics) {
+    this.baseInstantTime = baseInstantTime;
+    this.dataFileCommitTime = dataFileCommitTime;
+    this.deltaFilePaths = deltaFilePaths;
+    this.dataFilePath = dataFilePath;
+    this.id = new HoodieFileGroupId(partitionPath, fileId);
+    this.metrics = metrics;
+  }
+
   public CompactionOperation(java.util.Optional<HoodieDataFile> dataFile, String partitionPath,
       List<HoodieLogFile> logFiles, Map<String, Double> metrics) {
     if (dataFile.isPresent()) {
       this.baseInstantTime = dataFile.get().getCommitTime();
-      this.dataFilePath = Optional.of(dataFile.get().getPath());
+      this.dataFilePath = Option.of(dataFile.get().getPath());
       this.id = new HoodieFileGroupId(partitionPath, dataFile.get().getFileId());
-      this.dataFileCommitTime = Optional.of(dataFile.get().getCommitTime());
+      this.dataFileCommitTime = Option.of(dataFile.get().getCommitTime());
     } else {
       assert logFiles.size() > 0;
-      this.dataFilePath = Optional.absent();
+      this.dataFilePath = Option.empty();
       this.baseInstantTime = FSUtils.getBaseCommitTimeFromLogPath(logFiles.get(0).getPath());
       this.id = new HoodieFileGroupId(partitionPath, FSUtils.getFileIdFromLogPath(logFiles.get(0).getPath()));
-      this.dataFileCommitTime = Optional.absent();
+      this.dataFileCommitTime = Option.empty();
     }
 
     this.deltaFilePaths = logFiles.stream().map(s -> s.getPath().toString())
@@ -70,7 +82,7 @@ public class CompactionOperation implements Serializable {
     return baseInstantTime;
   }
 
-  public Optional<String> getDataFileCommitTime() {
+  public Option<String> getDataFileCommitTime() {
     return dataFileCommitTime;
   }
 
@@ -78,7 +90,7 @@ public class CompactionOperation implements Serializable {
     return deltaFilePaths;
   }
 
-  public Optional<String> getDataFilePath() {
+  public Option<String> getDataFilePath() {
     return dataFilePath;
   }
 
@@ -106,10 +118,45 @@ public class CompactionOperation implements Serializable {
   public static CompactionOperation convertFromAvroRecordInstance(HoodieCompactionOperation operation) {
     CompactionOperation op = new CompactionOperation();
     op.baseInstantTime = operation.getBaseInstantTime();
-    op.dataFilePath = Optional.fromNullable(operation.getDataFilePath());
+    op.dataFilePath = Option.ofNullable(operation.getDataFilePath());
+    op.dataFileCommitTime =
+        op.dataFilePath.map(p -> FSUtils.getCommitTime(new Path(p).getName()));
     op.deltaFilePaths = new ArrayList<>(operation.getDeltaFilePaths());
     op.id = new HoodieFileGroupId(operation.getPartitionPath(), operation.getFileId());
     op.metrics = operation.getMetrics() == null ? new HashMap<>() : new HashMap<>(operation.getMetrics());
     return op;
   }
+
+  @Override
+  public String toString() {
+    return "CompactionOperation{"
+        + "baseInstantTime='" + baseInstantTime + '\''
+        + ", dataFileCommitTime=" + dataFileCommitTime
+        + ", deltaFilePaths=" + deltaFilePaths
+        + ", dataFilePath=" + dataFilePath
+        + ", id='" + id + '\''
+        + ", metrics=" + metrics
+        + '}';
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    CompactionOperation operation = (CompactionOperation) o;
+    return Objects.equals(baseInstantTime, operation.baseInstantTime)
+        && Objects.equals(dataFileCommitTime, operation.dataFileCommitTime)
+        && Objects.equals(deltaFilePaths, operation.deltaFilePaths)
+        && Objects.equals(dataFilePath, operation.dataFilePath)
+        && Objects.equals(id, operation.id);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(baseInstantTime, id);
+  }
 }
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/FileSlice.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/FileSlice.java
index 30ef2dc..9393c83 100644
--- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/FileSlice.java
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/FileSlice.java
@@ -19,6 +19,7 @@
 package com.uber.hoodie.common.model;
 
 import java.io.Serializable;
+import java.util.Objects;
 import java.util.Optional;
 import java.util.TreeSet;
 import java.util.stream.Stream;
@@ -93,14 +94,42 @@ public class FileSlice implements Serializable {
     return Optional.ofNullable(dataFile);
   }
 
+  /**
+   * Returns true if there is no data file and no log files. Happens as part of pending compaction
+   * @return
+   */
+  public boolean isEmpty() {
+    return (dataFile == null) && (logFiles.isEmpty());
+  }
+
   @Override
   public String toString() {
     final StringBuilder sb = new StringBuilder("FileSlice {");
     sb.append("fileGroupId=").append(fileGroupId);
-    sb.append("baseCommitTime=").append(baseInstantTime);
+    sb.append(", baseCommitTime=").append(baseInstantTime);
     sb.append(", dataFile='").append(dataFile).append('\'');
     sb.append(", logFiles='").append(logFiles).append('\'');
     sb.append('}');
     return sb.toString();
   }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    FileSlice slice = (FileSlice) o;
+    return Objects.equals(fileGroupId, slice.fileGroupId)
+        && Objects.equals(baseInstantTime, slice.baseInstantTime)
+        && Objects.equals(dataFile, slice.dataFile)
+        && Objects.equals(logFiles, slice.logFiles);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(fileGroupId, baseInstantTime);
+  }
 }
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieDataFile.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieDataFile.java
index baa3c75..8e958c9 100644
--- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieDataFile.java
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieDataFile.java
@@ -18,53 +18,82 @@ package com.uber.hoodie.common.model;
 
 import com.uber.hoodie.common.util.FSUtils;
 import java.io.Serializable;
-import java.util.Comparator;
+import java.util.Objects;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
 
 public class HoodieDataFile implements Serializable {
 
-  private FileStatus fileStatus;
+  private transient FileStatus fileStatus;
+  private final String fullPath;
+  private long fileLen;
 
   public HoodieDataFile(FileStatus fileStatus) {
     this.fileStatus = fileStatus;
+    this.fullPath = fileStatus.getPath().toString();
+    this.fileLen = fileStatus.getLen();
+  }
+
+  public HoodieDataFile(String filePath) {
+    this.fileStatus = null;
+    this.fullPath = filePath;
+    this.fileLen = -1;
   }
 
   public String getFileId() {
-    return FSUtils.getFileId(fileStatus.getPath().getName());
+    return FSUtils.getFileId(getFileName());
   }
 
   public String getCommitTime() {
-    return FSUtils.getCommitTime(fileStatus.getPath().getName());
+    return FSUtils.getCommitTime(getFileName());
   }
 
   public String getPath() {
-    return fileStatus.getPath().toString();
+    return fullPath;
   }
 
   public String getFileName() {
-    return fileStatus.getPath().getName();
+    return new Path(fullPath).getName();
   }
 
   public FileStatus getFileStatus() {
     return fileStatus;
   }
 
-  public static Comparator<HoodieDataFile> getCommitTimeComparator() {
-    return (o1, o2) -> {
-      // reverse the order
-      return o2.getCommitTime().compareTo(o1.getCommitTime());
-    };
+  public long getFileSize() {
+    return fileLen;
+  }
+
+  public void setFileLen(long fileLen) {
+    this.fileLen = fileLen;
   }
 
-  public long getFileSize() {
-    return fileStatus.getLen();
+  public long getFileLen() {
+    return fileLen;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    HoodieDataFile dataFile = (HoodieDataFile) o;
+    return Objects.equals(fullPath, dataFile.fullPath);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(fullPath);
   }
 
   @Override
   public String toString() {
-    final StringBuilder sb = new StringBuilder("HoodieDataFile {");
-    sb.append("fileStatus=").append(fileStatus);
-    sb.append('}');
-    return sb.toString();
+    return "HoodieDataFile{"
+        + "fullPath=" + fullPath
+        + ", fileLen=" + fileLen
+        + '}';
   }
 }
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieFileGroup.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieFileGroup.java
index 798a671..fb0621a 100644
--- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieFileGroup.java
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieFileGroup.java
@@ -20,6 +20,7 @@ package com.uber.hoodie.common.model;
 
 import com.uber.hoodie.common.table.HoodieTimeline;
 import com.uber.hoodie.common.table.timeline.HoodieInstant;
+import com.uber.hoodie.common.util.Option;
 import java.io.Serializable;
 import java.util.*;
 import java.util.stream.Stream;
@@ -51,7 +52,7 @@ public class HoodieFileGroup implements Serializable {
   /**
    * The last completed instant, that acts as a high watermark for all getters
    */
-  private final Optional<HoodieInstant> lastInstant;
+  private final Option<HoodieInstant> lastInstant;
 
   public HoodieFileGroup(String partitionPath, String id, HoodieTimeline timeline) {
     this(new HoodieFileGroupId(partitionPath, id), timeline);
@@ -61,7 +62,7 @@ public class HoodieFileGroup implements Serializable {
     this.fileGroupId = fileGroupId;
     this.fileSlices = new TreeMap<>(HoodieFileGroup.getReverseCommitTimeComparator());
     this.timeline = timeline;
-    this.lastInstant = timeline.lastInstant();
+    this.lastInstant = Option.fromJavaOptional(timeline.lastInstant());
   }
 
   /**
@@ -152,6 +153,13 @@ public class HoodieFileGroup implements Serializable {
   }
 
   /**
+   * Gets the latest data file
+   */
+  public Optional<HoodieDataFile> getLatestDataFile() {
+    return getAllDataFiles().findFirst();
+  }
+
+  /**
    * Obtain the latest file slice, upto a commitTime i.e <= maxCommitTime
    */
   public Optional<FileSlice> getLatestFileSliceBeforeOrOn(String maxCommitTime) {
@@ -197,7 +205,20 @@ public class HoodieFileGroup implements Serializable {
     final StringBuilder sb = new StringBuilder("HoodieFileGroup {");
     sb.append("id=").append(fileGroupId);
     sb.append(", fileSlices='").append(fileSlices).append('\'');
+    sb.append(", lastInstant='").append(lastInstant).append('\'');
     sb.append('}');
     return sb.toString();
   }
+
+  public void addFileSlice(FileSlice slice) {
+    fileSlices.put(slice.getBaseInstantTime(), slice);
+  }
+
+  public Stream<FileSlice> getAllRawFileSlices() {
+    return fileSlices.values().stream();
+  }
+
+  public HoodieTimeline getTimeline() {
+    return timeline;
+  }
 }
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieLogFile.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieLogFile.java
index 88acc4a..0820b57 100644
--- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieLogFile.java
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieLogFile.java
@@ -22,14 +22,14 @@ import com.uber.hoodie.common.util.FSUtils;
 import java.io.IOException;
 import java.io.Serializable;
 import java.util.Comparator;
-import java.util.Optional;
+import java.util.Objects;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 
 /**
- * Abstracts a single log file. Contains methods to extract metadata like the fileId, version and
- * extension from the log file path.
+ * Abstracts a single log file. Contains methods to extract metadata like the fileId, version and extension from the log
+ * file path.
  * <p>
  * Also contains logic to roll-over the log file
  */
@@ -38,54 +38,68 @@ public class HoodieLogFile implements Serializable {
   public static final String DELTA_EXTENSION = ".log";
   public static final Integer LOGFILE_BASE_VERSION = 1;
 
-  private final Path path;
-  private Optional<FileStatus> fileStatus;
+  private transient FileStatus fileStatus;
+  private final String pathStr;
+  private long fileLen;
 
   public HoodieLogFile(FileStatus fileStatus) {
-    this(fileStatus.getPath());
-    this.fileStatus = Optional.of(fileStatus);
+    this.fileStatus = fileStatus;
+    this.pathStr = fileStatus.getPath().toString();
+    this.fileLen = fileStatus.getLen();
   }
 
   public HoodieLogFile(Path logPath) {
-    this.path = logPath;
-    this.fileStatus = Optional.empty();
+    this.fileStatus = null;
+    this.pathStr = logPath.toString();
+    this.fileLen = 0;
+  }
+
+  public HoodieLogFile(String logPathStr) {
+    this.fileStatus = null;
+    this.pathStr = logPathStr;
+    this.fileLen = -1;
   }
 
   public String getFileId() {
-    return FSUtils.getFileIdFromLogPath(path);
+    return FSUtils.getFileIdFromLogPath(getPath());
   }
 
   public String getBaseCommitTime() {
-    return FSUtils.getBaseCommitTimeFromLogPath(path);
+    return FSUtils.getBaseCommitTimeFromLogPath(getPath());
   }
 
   public int getLogVersion() {
-    return FSUtils.getFileVersionFromLog(path);
+    return FSUtils.getFileVersionFromLog(getPath());
   }
 
   public String getFileExtension() {
-    return FSUtils.getFileExtensionFromLog(path);
+    return FSUtils.getFileExtensionFromLog(getPath());
   }
 
   public Path getPath() {
-    return path;
+    return new Path(pathStr);
   }
 
   public String getFileName() {
-    return path.getName();
+    return getPath().getName();
   }
 
-  public Optional<FileStatus> getFileStatus() {
-    return fileStatus;
+  public void setFileLen(long fileLen) {
+    this.fileLen = fileLen;
+  }
+
+  public long getFileSize() {
+    return fileLen;
   }
 
-  public Optional<Long> getFileSize() {
-    return fileStatus.map(FileStatus::getLen);
+  public FileStatus getFileStatus() {
+    return fileStatus;
   }
 
   public HoodieLogFile rollOver(FileSystem fs) throws IOException {
     String fileId = getFileId();
     String baseCommitTime = getBaseCommitTime();
+    Path path = getPath();
     String extension = "." + FSUtils.getFileExtensionFromLog(path);
     int newVersion = FSUtils
         .computeNextLogVersion(fs, path.getParent(), fileId,
@@ -95,7 +109,16 @@ public class HoodieLogFile implements Serializable {
   }
 
   public static Comparator<HoodieLogFile> getBaseInstantAndLogVersionComparator() {
-    return (o1, o2) -> {
+    return new BaseInstantAndLogVersionComparator();
+  }
+
+  /**
+   * Comparator to order log-files
+   */
+  private static class BaseInstantAndLogVersionComparator implements Comparator<HoodieLogFile>, Serializable {
+
+    @Override
+    public int compare(HoodieLogFile o1, HoodieLogFile o2) {
       String baseInstantTime1 = o1.getBaseCommitTime();
       String baseInstantTime2 = o2.getBaseCommitTime();
       if (baseInstantTime1.equals(baseInstantTime2)) {
@@ -104,7 +127,7 @@ public class HoodieLogFile implements Serializable {
       }
       // reverse the order by base-commits
       return baseInstantTime2.compareTo(baseInstantTime1);
-    };
+    }
   }
 
   @Override
@@ -116,16 +139,19 @@ public class HoodieLogFile implements Serializable {
       return false;
     }
     HoodieLogFile that = (HoodieLogFile) o;
-    return path != null ? path.equals(that.path) : that.path == null;
+    return Objects.equals(pathStr, that.pathStr);
   }
 
   @Override
   public int hashCode() {
-    return path != null ? path.hashCode() : 0;
+    return Objects.hash(pathStr);
   }
 
   @Override
   public String toString() {
-    return "HoodieLogFile {" + path + '}';
+    return "HoodieLogFile{"
+        + "pathStr='" + pathStr + '\''
+        + ", fileLen=" + fileLen
+        + '}';
   }
 }
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieWriteStat.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieWriteStat.java
index 7523684..e74606c 100644
--- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieWriteStat.java
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieWriteStat.java
@@ -133,6 +133,11 @@ public class HoodieWriteStat implements Serializable {
    */
   private long totalRollbackBlocks;
 
+  /**
+   * File Size as of close
+   */
+  private long fileSizeInBytes;
+
   @Nullable
   @JsonIgnore
   private RuntimeStats runtimeStats;
@@ -285,6 +290,14 @@ public class HoodieWriteStat implements Serializable {
     this.totalRollbackBlocks = totalRollbackBlocks;
   }
 
+  public long getFileSizeInBytes() {
+    return fileSizeInBytes;
+  }
+
+  public void setFileSizeInBytes(long fileSizeInBytes) {
+    this.fileSizeInBytes = fileSizeInBytes;
+  }
+
   @Nullable
   public RuntimeStats getRuntimeStats() {
     return runtimeStats;
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableConfig.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableConfig.java
index 345f290..6779e6b 100644
--- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableConfig.java
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableConfig.java
@@ -77,6 +77,10 @@ public class HoodieTableConfig implements Serializable {
     this.props = props;
   }
 
+  public HoodieTableConfig(Properties props) {
+    this.props = props;
+  }
+
   /**
    * For serailizing and de-serializing
    *
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java
index a2c9906..7570786 100644
--- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTableMetaClient.java
@@ -195,6 +195,16 @@ public class HoodieTableMetaClient implements Serializable {
   }
 
   /**
+   * Reload ActiveTimeline and cache
+   *
+   * @return Active instants timeline
+   */
+  public synchronized HoodieActiveTimeline reloadActiveTimeline() {
+    activeTimeline = new HoodieActiveTimeline(this);
+    return activeTimeline;
+  }
+
+  /**
    * Get the archived commits as a timeline. This is costly operation, as all data from the archived
    * files are read. This should not be used, unless for historical debugging purposes
    *
@@ -406,4 +416,20 @@ public class HoodieTableMetaClient implements Serializable {
     sb.append('}');
     return sb.toString();
   }
+
+  public void setBasePath(String basePath) {
+    this.basePath = basePath;
+  }
+
+  public void setMetaPath(String metaPath) {
+    this.metaPath = metaPath;
+  }
+
+  public void setActiveTimeline(HoodieActiveTimeline activeTimeline) {
+    this.activeTimeline = activeTimeline;
+  }
+
+  public void setTableConfig(HoodieTableConfig tableConfig) {
+    this.tableConfig = tableConfig;
+  }
 }
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTimeline.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTimeline.java
index 6c8a204..509de9c 100644
--- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTimeline.java
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTimeline.java
@@ -23,6 +23,7 @@ import com.uber.hoodie.common.util.StringUtils;
 import java.io.Serializable;
 import java.util.Optional;
 import java.util.function.BiPredicate;
+import java.util.function.Predicate;
 import java.util.stream.Stream;
 
 /**
@@ -70,6 +71,8 @@ public interface HoodieTimeline extends Serializable {
   String INFLIGHT_RESTORE_EXTENSION = "." + RESTORE_ACTION + INFLIGHT_EXTENSION;
   String RESTORE_EXTENSION = "." + RESTORE_ACTION;
 
+  String INVALID_INSTANT_TS  = "0";
+
   /**
    * Filter this timeline to just include the in-flights
    *
@@ -117,6 +120,11 @@ public interface HoodieTimeline extends Serializable {
   HoodieTimeline findInstantsAfter(String commitTime, int numCommits);
 
   /**
+   * Custom Filter of Instants
+   */
+  HoodieTimeline filter(Predicate<HoodieInstant> filter);
+
+  /**
    * If the timeline has any instants
    *
    * @return true if timeline is empty
@@ -143,6 +151,13 @@ public interface HoodieTimeline extends Serializable {
    */
   Optional<HoodieInstant> lastInstant();
 
+
+  /**
+   * Get hash of timeline
+   * @return
+   */
+  String getTimelineHash();
+
   /**
    * @return nth completed instant going back from the last completed instant
    */
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/SyncableFileSystemView.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/SyncableFileSystemView.java
new file mode 100644
index 0000000..378a5da
--- /dev/null
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/SyncableFileSystemView.java
@@ -0,0 +1,44 @@
+/*
+ *  Copyright (c) 2019 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  You may obtain a copy of the License at
+ *
+ *           http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package com.uber.hoodie.common.table;
+
+/*
+ * A consolidated file-system view interface exposing both realtime and read-optimized views along with
+ * update operations.
+ */
+public interface SyncableFileSystemView extends TableFileSystemView, TableFileSystemView.ReadOptimizedView,
+    TableFileSystemView.RealtimeView {
+
+
+
+  /**
+   * Allow View to release resources and close
+   */
+  void close();
+
+  /**
+   * Reset View so that they can be refreshed
+   */
+  void reset();
+
+  /**
+   * Read the latest timeline and refresh the file-system view to match the current state of the file-system.
+   * The refresh can either be done incrementally (from reading file-slices in metadata files) or from scratch by
+   * reseting view storage
+   */
+  void sync();
+}
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/TableFileSystemView.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/TableFileSystemView.java
index d0f2d87..994d9d3 100644
--- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/TableFileSystemView.java
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/TableFileSystemView.java
@@ -16,9 +16,13 @@
 
 package com.uber.hoodie.common.table;
 
+import com.uber.hoodie.common.model.CompactionOperation;
 import com.uber.hoodie.common.model.FileSlice;
 import com.uber.hoodie.common.model.HoodieDataFile;
 import com.uber.hoodie.common.model.HoodieFileGroup;
+import com.uber.hoodie.common.table.timeline.HoodieInstant;
+import com.uber.hoodie.common.util.Option;
+import com.uber.hoodie.common.util.collection.Pair;
 import java.util.List;
 import java.util.stream.Stream;
 
@@ -30,9 +34,9 @@ import java.util.stream.Stream;
 public interface TableFileSystemView {
 
   /**
-   * ReadOptimizedView - methods to provide a view of columnar data files only.
+   * ReadOptimizedView with methods to only access latest version of file for the instant(s) passed.
    */
-  interface ReadOptimizedView {
+  interface ReadOptimizedViewWithLatestSlice {
 
     /**
      * Stream all the latest data files in the given partition
@@ -40,39 +44,48 @@ public interface TableFileSystemView {
     Stream<HoodieDataFile> getLatestDataFiles(String partitionPath);
 
     /**
+     * Get Latest data file for a partition and file-Id
+     */
+    Option<HoodieDataFile> getLatestDataFile(String partitionPath, String fileId);
+
+    /**
      * Stream all the latest data files, in the file system view
      */
     Stream<HoodieDataFile> getLatestDataFiles();
 
     /**
-     * Stream all the latest version data files in the given partition with precondition that
-     * commitTime(file) before maxCommitTime
+     * Stream all the latest version data files in the given partition with precondition that commitTime(file) before
+     * maxCommitTime
      */
     Stream<HoodieDataFile> getLatestDataFilesBeforeOrOn(String partitionPath,
         String maxCommitTime);
 
     /**
-     * Stream all the latest version data files in the given partition with precondition that
-     * instant time of file matches passed in instant time.
-     */
-    Stream<HoodieDataFile> getLatestDataFilesOn(String partitionPath, String instantTime);
-
-    /**
      * Stream all the latest data files pass
      */
     Stream<HoodieDataFile> getLatestDataFilesInRange(List<String> commitsToReturn);
+  }
 
+  /**
+   * ReadOptimizedView - methods to provide a view of columnar data files only.
+   */
+  interface ReadOptimizedView extends ReadOptimizedViewWithLatestSlice {
     /**
      * Stream all the data file versions grouped by FileId for a given partition
      */
     Stream<HoodieDataFile> getAllDataFiles(String partitionPath);
+
+    /**
+     * Get the version of data file matching the instant time in the given partition
+     */
+    Option<HoodieDataFile> getDataFileOn(String partitionPath, String instantTime, String fileId);
+
   }
 
   /**
-   * RealtimeView - methods to access a combination of columnar data files + log files with real
-   * time data.
+   * RealtimeView with methods to only access latest version of file-slice for the instant(s) passed.
    */
-  interface RealtimeView {
+  interface RealtimeViewWithLatestSlice {
 
     /**
      * Stream all the latest file slices in the given partition
@@ -80,6 +93,11 @@ public interface TableFileSystemView {
     Stream<FileSlice> getLatestFileSlices(String partitionPath);
 
     /**
+     * Get Latest File Slice for a given fileId in a given partition
+     */
+    Option<FileSlice> getLatestFileSlice(String partitionPath, String fileId);
+
+    /**
      * Stream all the latest uncompacted file slices in the given partition
      */
     Stream<FileSlice> getLatestUnCompactedFileSlices(String partitionPath);
@@ -106,15 +124,39 @@ public interface TableFileSystemView {
      * Stream all the latest file slices, in the given range
      */
     Stream<FileSlice> getLatestFileSliceInRange(List<String> commitsToReturn);
+  }
+
+  /**
+   * RealtimeView - methods to access a combination of columnar data files + log files with real time data.
+   */
+  interface RealtimeView extends RealtimeViewWithLatestSlice {
 
     /**
      * Stream all the file slices for a given partition, latest or not.
      */
     Stream<FileSlice> getAllFileSlices(String partitionPath);
+
   }
 
   /**
    * Stream all the file groups for a given partition
    */
   Stream<HoodieFileGroup> getAllFileGroups(String partitionPath);
+
+  /**
+   * Return Pending Compaction Operations
+   *
+   * @return Pair<Pair<InstantTime,CompactionOperation>>
+   */
+  Stream<Pair<String, CompactionOperation>> getPendingCompactionOperations();
+
+  /**
+   * Last Known Instant on which the view is built
+   */
+  Option<HoodieInstant> getLastInstant();
+
+  /**
+   * Timeline corresponding to the view
+   */
+  HoodieTimeline getTimeline();
 }
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieActiveTimeline.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieActiveTimeline.java
index f81f7d1..d34012e 100644
--- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieActiveTimeline.java
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieActiveTimeline.java
@@ -17,6 +17,7 @@
 package com.uber.hoodie.common.table.timeline;
 
 import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Sets;
 import com.uber.hoodie.common.table.HoodieTableMetaClient;
 import com.uber.hoodie.common.table.HoodieTimeline;
@@ -58,7 +59,7 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
           INFLIGHT_RESTORE_EXTENSION, RESTORE_EXTENSION}));
 
   private static final transient Logger log = LogManager.getLogger(HoodieActiveTimeline.class);
-  private HoodieTableMetaClient metaClient;
+  protected HoodieTableMetaClient metaClient;
 
   /**
    * Returns next commit time in the {@link #COMMIT_FORMATTER} format.
@@ -71,9 +72,9 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
     // Filter all the filter in the metapath and include only the extensions passed and
     // convert them into HoodieInstant
     try {
-      this.instants = HoodieTableMetaClient.scanHoodieInstantsFromFileSystem(metaClient.getFs(),
-          new Path(metaClient.getMetaPath()), includedExtensions);
-      log.info("Loaded instants " + instants);
+      this.setInstants(HoodieTableMetaClient.scanHoodieInstantsFromFileSystem(metaClient.getFs(),
+          new Path(metaClient.getMetaPath()), includedExtensions));
+      log.info("Loaded instants " + getInstants());
     } catch (IOException e) {
       throw new HoodieIOException("Failed to scan metadata", e);
     }
@@ -85,7 +86,9 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
   }
 
   public HoodieActiveTimeline(HoodieTableMetaClient metaClient) {
-    this(metaClient, VALID_EXTENSIONS_IN_ACTIVE_TIMELINE);
+    this(metaClient,
+        new ImmutableSet.Builder<String>()
+            .addAll(VALID_EXTENSIONS_IN_ACTIVE_TIMELINE).build());
   }
 
   /**
@@ -158,7 +161,7 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
    * @param actions actions allowed in the timeline
    */
   public HoodieTimeline getTimelineOfActions(Set<String> actions) {
-    return new HoodieDefaultTimeline(instants.stream().filter(s -> actions.contains(s.getAction())),
+    return new HoodieDefaultTimeline(getInstants().filter(s -> actions.contains(s.getAction())),
         (Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails);
   }
 
@@ -195,9 +198,8 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
         (Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails);
   }
 
-
   protected Stream<HoodieInstant> filterInstantsByAction(String action) {
-    return instants.stream().filter(s -> s.getAction().equals(action));
+    return getInstants().filter(s -> s.getAction().equals(action));
   }
 
   public void createInflight(HoodieInstant instant) {
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieArchivedTimeline.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieArchivedTimeline.java
index 1da29f6..ef24310 100644
--- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieArchivedTimeline.java
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieArchivedTimeline.java
@@ -61,9 +61,9 @@ public class HoodieArchivedTimeline extends HoodieDefaultTimeline {
         // This is okay because only tooling will load the archived commit timeline today
         readCommits.put(key.toString(), Arrays.copyOf(val.getBytes(), val.getLength()));
       }
-      this.instants = readCommits.keySet().stream().map(
+      this.setInstants(readCommits.keySet().stream().map(
           s -> new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, s)).collect(
-          Collectors.toList());
+          Collectors.toList()));
     } catch (IOException e) {
       throw new HoodieIOException(
           "Could not load archived commit timeline from path " + archiveLogPath, e);
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieDefaultTimeline.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieDefaultTimeline.java
index 87c664c..0f89bbe 100644
--- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieDefaultTimeline.java
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieDefaultTimeline.java
@@ -17,11 +17,17 @@
 package com.uber.hoodie.common.table.timeline;
 
 import com.uber.hoodie.common.table.HoodieTimeline;
+import com.uber.hoodie.common.util.StringUtils;
+import com.uber.hoodie.exception.HoodieException;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
 import java.util.List;
 import java.util.Optional;
 import java.util.function.Function;
+import java.util.function.Predicate;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
+import org.apache.commons.codec.binary.Hex;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
 
@@ -36,13 +42,29 @@ public class HoodieDefaultTimeline implements HoodieTimeline {
 
   private static final transient Logger log = LogManager.getLogger(HoodieDefaultTimeline.class);
 
-  protected Function<HoodieInstant, Optional<byte[]>> details;
-  protected List<HoodieInstant> instants;
+  private static final String HASHING_ALGORITHM = "SHA-256";
+
+  protected transient Function<HoodieInstant, Optional<byte[]>> details;
+  private List<HoodieInstant> instants;
+  private String timelineHash;
 
   public HoodieDefaultTimeline(Stream<HoodieInstant> instants,
       Function<HoodieInstant, Optional<byte[]>> details) {
-    this.instants = instants.collect(Collectors.toList());
     this.details = details;
+    setInstants(instants.collect(Collectors.toList()));
+  }
+
+  public void setInstants(List<HoodieInstant> instants) {
+    this.instants = instants;
+    final MessageDigest md;
+    try {
+      md = MessageDigest.getInstance(HASHING_ALGORITHM);
+      this.instants.stream().forEach(i -> md.update(
+          StringUtils.joinUsingDelim("_", i.getTimestamp(), i.getAction(), i.getState().name()).getBytes()));
+    } catch (NoSuchAlgorithmException nse) {
+      throw new HoodieException(nse);
+    }
+    this.timelineHash = new String(Hex.encodeHex(md.digest()));
   }
 
   /**
@@ -102,6 +124,11 @@ public class HoodieDefaultTimeline implements HoodieTimeline {
   }
 
   @Override
+  public HoodieTimeline filter(Predicate<HoodieInstant> filter) {
+    return new HoodieDefaultTimeline(instants.stream().filter(filter), details);
+  }
+
+  @Override
   public boolean empty() {
     return !instants.stream().findFirst().isPresent();
   }
@@ -149,6 +176,11 @@ public class HoodieDefaultTimeline implements HoodieTimeline {
   }
 
   @Override
+  public String getTimelineHash() {
+    return timelineHash;
+  }
+
+  @Override
   public Stream<HoodieInstant> getInstants() {
     return instants.stream();
   }
@@ -160,7 +192,6 @@ public class HoodieDefaultTimeline implements HoodieTimeline {
         && HoodieTimeline.compareTimestamps(instant, firstCommit.get().getTimestamp(), LESSER);
   }
 
-
   @Override
   public Optional<byte[]> getInstantDetails(HoodieInstant instant) {
     return details.apply(instant);
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieInstant.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieInstant.java
index f24eb9d..4168471 100644
--- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieInstant.java
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieInstant.java
@@ -39,7 +39,9 @@ public class HoodieInstant implements Serializable {
     // Inflight instant
     INFLIGHT,
     // Committed instant
-    COMPLETED
+    COMPLETED,
+    // Invalid instant
+    INVALID
   }
 
   private State state = State.COMPLETED;
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/dto/CompactionOpDTO.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/dto/CompactionOpDTO.java
new file mode 100644
index 0000000..d35876e
--- /dev/null
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/dto/CompactionOpDTO.java
@@ -0,0 +1,75 @@
+/*
+ * Copyright (c) 2019 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *          http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.uber.hoodie.common.table.timeline.dto;
+
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.uber.hoodie.common.model.CompactionOperation;
+import com.uber.hoodie.common.util.Option;
+import com.uber.hoodie.common.util.collection.Pair;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class CompactionOpDTO {
+
+  @JsonProperty("baseInstant")
+  String baseInstantTime;
+
+  @JsonProperty("compactionInstant")
+  String compactionInstantTime;
+
+  @JsonProperty("dataFileInstant")
+  private String dataFileCommitTime;
+
+  @JsonProperty("deltaFiles")
+  private List<String> deltaFilePaths;
+
+  @JsonProperty("baseFile")
+  private String dataFilePath;
+
+  @JsonProperty("id")
+  private String fileId;
+
+  @JsonProperty("partition")
+  private String partitionPath;
+
+  @JsonProperty("metrics")
+  private Map<String, Double> metrics;
+
+  public static CompactionOpDTO fromCompactionOperation(String compactionInstantTime,
+      CompactionOperation op) {
+    CompactionOpDTO dto = new CompactionOpDTO();
+    dto.fileId = op.getFileId();
+    dto.compactionInstantTime = compactionInstantTime;
+    dto.baseInstantTime = op.getBaseInstantTime();
+    dto.dataFileCommitTime = op.getDataFileCommitTime().orElse(null);
+    dto.dataFilePath = op.getDataFilePath().orElse(null);
+    dto.deltaFilePaths = new ArrayList<>(op.getDeltaFilePaths());
+    dto.partitionPath = op.getPartitionPath();
+    dto.metrics = op.getMetrics() == null ? new HashMap<>() : new HashMap<>(op.getMetrics());
+    return dto;
+  }
+
+  public static Pair<String, CompactionOperation> toCompactionOperation(CompactionOpDTO dto) {
+    return Pair.of(dto.compactionInstantTime, new CompactionOperation(dto.fileId, dto.partitionPath,
+        dto.baseInstantTime, Option.ofNullable(dto.dataFileCommitTime), dto.deltaFilePaths,
+        Option.ofNullable(dto.dataFilePath), dto.metrics));
+  }
+}
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/dto/DataFileDTO.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/dto/DataFileDTO.java
new file mode 100644
index 0000000..aa16e42
--- /dev/null
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/dto/DataFileDTO.java
@@ -0,0 +1,60 @@
+/*
+ * Copyright (c) 2019 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *          http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.uber.hoodie.common.table.timeline.dto;
+
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.uber.hoodie.common.model.HoodieDataFile;
+
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class DataFileDTO {
+
+  @JsonProperty("fileStatus")
+  private FileStatusDTO fileStatus;
+  @JsonProperty("fullPath")
+  private String fullPath;
+  @JsonProperty("fileLen")
+  private long fileLen;
+
+  public static HoodieDataFile toHoodieDataFile(DataFileDTO dto) {
+    if (null == dto) {
+      return null;
+    }
+
+    HoodieDataFile dataFile = null;
+    if (null != dto.fileStatus) {
+      dataFile = new HoodieDataFile(FileStatusDTO.toFileStatus(dto.fileStatus));
+    } else {
+      dataFile = new HoodieDataFile(dto.fullPath);
+      dataFile.setFileLen(dto.fileLen);
+    }
+    return dataFile;
+  }
+
+  public static DataFileDTO fromHoodieDataFile(HoodieDataFile dataFile) {
+    if (null == dataFile) {
+      return null;
+    }
+
+    DataFileDTO dto = new DataFileDTO();
+    dto.fileStatus = FileStatusDTO.fromFileStatus(dataFile.getFileStatus());
+    dto.fullPath = dataFile.getPath();
+    dto.fileLen = dataFile.getFileLen();
+    return dto;
+  }
+
+}
\ No newline at end of file
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/dto/FSPermissionDTO.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/dto/FSPermissionDTO.java
new file mode 100644
index 0000000..92be305
--- /dev/null
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/dto/FSPermissionDTO.java
@@ -0,0 +1,61 @@
+/*
+ * Copyright (c) 2019 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *          http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.uber.hoodie.common.table.timeline.dto;
+
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import java.io.Serializable;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+
+/**
+ * A serializable FS Permission
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class FSPermissionDTO implements Serializable {
+
+  @JsonProperty("useraction")
+  FsAction useraction;
+
+  @JsonProperty("groupaction")
+  FsAction groupaction;
+
+  @JsonProperty("otheraction")
+  FsAction otheraction;
+
+  @JsonProperty("stickyBit")
+  boolean stickyBit;
+
+  public static FSPermissionDTO fromFsPermission(FsPermission permission) {
+    if (null == permission) {
+      return null;
+    }
+    FSPermissionDTO dto = new FSPermissionDTO();
+    dto.useraction = permission.getUserAction();
+    dto.groupaction = permission.getGroupAction();
+    dto.otheraction = permission.getOtherAction();
+    dto.stickyBit = permission.getStickyBit();
+    return dto;
+  }
+
+  public static FsPermission fromFsPermissionDTO(FSPermissionDTO dto) {
+    if (null == dto) {
+      return null;
+    }
+    return new FsPermission(dto.useraction, dto.groupaction, dto.otheraction, dto.stickyBit);
+  }
+}
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/dto/FileGroupDTO.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/dto/FileGroupDTO.java
new file mode 100644
index 0000000..9049485
--- /dev/null
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/dto/FileGroupDTO.java
@@ -0,0 +1,56 @@
+/*
+ * Copyright (c) 2019 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *          http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.uber.hoodie.common.table.timeline.dto;
+
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.uber.hoodie.common.model.HoodieFileGroup;
+import com.uber.hoodie.common.table.HoodieTableMetaClient;
+import java.util.List;
+import java.util.stream.Collectors;
+
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class FileGroupDTO {
+
+  @JsonProperty("partition")
+  String partition;
+
+  @JsonProperty("fileId")
+  String id;
+
+  @JsonProperty("slices")
+  List<FileSliceDTO> slices;
+
+  @JsonProperty("timeline")
+  TimelineDTO timeline;
+
+  public static FileGroupDTO fromFileGroup(HoodieFileGroup fileGroup) {
+    FileGroupDTO dto = new FileGroupDTO();
+    dto.partition = fileGroup.getPartitionPath();
+    dto.id = fileGroup.getFileGroupId().getFileId();
+    dto.slices = fileGroup.getAllRawFileSlices().map(FileSliceDTO::fromFileSlice).collect(Collectors.toList());
+    dto.timeline = TimelineDTO.fromTimeline(fileGroup.getTimeline());
+    return dto;
+  }
+
+  public static HoodieFileGroup toFileGroup(FileGroupDTO dto, HoodieTableMetaClient metaClient) {
+    HoodieFileGroup fileGroup = new HoodieFileGroup(dto.partition, dto.id,
+        TimelineDTO.toTimeline(dto.timeline, metaClient));
+    dto.slices.stream().map(FileSliceDTO::toFileSlice).forEach(fileSlice -> fileGroup.addFileSlice(fileSlice));
+    return fileGroup;
+  }
+}
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/dto/FilePathDTO.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/dto/FilePathDTO.java
new file mode 100644
index 0000000..16c510d
--- /dev/null
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/dto/FilePathDTO.java
@@ -0,0 +1,51 @@
+/*
+ * Copyright (c) 2019 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *          http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.uber.hoodie.common.table.timeline.dto;
+
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import java.net.URI;
+import java.net.URISyntaxException;
+import org.apache.hadoop.fs.Path;
+
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class FilePathDTO {
+
+  @JsonProperty("uri")
+  private String uri;
+
+  public static FilePathDTO fromPath(Path path) {
+    if (null == path) {
+      return null;
+    }
+    FilePathDTO dto = new FilePathDTO();
+    dto.uri = path.toUri().toString();
+    return dto;
+  }
+
+  public static Path toPath(FilePathDTO dto) {
+    if (null == dto) {
+      return null;
+    }
+
+    try {
+      return new Path(new URI(dto.uri));
+    } catch (URISyntaxException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/dto/FileSliceDTO.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/dto/FileSliceDTO.java
new file mode 100644
index 0000000..34908a6
--- /dev/null
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/dto/FileSliceDTO.java
@@ -0,0 +1,55 @@
+/*
+ * Copyright (c) 2019 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *          http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.uber.hoodie.common.table.timeline.dto;
+
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.uber.hoodie.common.model.FileSlice;
+import java.util.List;
+import java.util.stream.Collectors;
+
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class FileSliceDTO {
+
+  @JsonProperty("dataFile")
+  DataFileDTO dataFile;
+  @JsonProperty("logFiles")
+  List<LogFileDTO> logFiles;
+  @JsonProperty("partition")
+  private String partitionPath;
+  @JsonProperty("fileId")
+  private String fileId;
+  @JsonProperty("baseInstant")
+  private String baseInstantTime;
+
+  public static FileSliceDTO fromFileSlice(FileSlice slice) {
+    FileSliceDTO dto = new FileSliceDTO();
+    dto.partitionPath = slice.getPartitionPath();
+    dto.baseInstantTime = slice.getBaseInstantTime();
+    dto.fileId = slice.getFileId();
+    dto.dataFile = slice.getDataFile().map(DataFileDTO::fromHoodieDataFile).orElse(null);
+    dto.logFiles = slice.getLogFiles().map(LogFileDTO::fromHoodieLogFile).collect(Collectors.toList());
+    return dto;
+  }
+
+  public static FileSlice toFileSlice(FileSliceDTO dto) {
+    FileSlice slice = new FileSlice(dto.partitionPath, dto.baseInstantTime, dto.fileId);
+    slice.setDataFile(DataFileDTO.toHoodieDataFile(dto.dataFile));
+    dto.logFiles.stream().forEach(lf -> slice.addLogFile(LogFileDTO.toHoodieLogFile(lf)));
+    return slice;
+  }
+}
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/dto/FileStatusDTO.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/dto/FileStatusDTO.java
new file mode 100644
index 0000000..c09a5d2
--- /dev/null
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/dto/FileStatusDTO.java
@@ -0,0 +1,97 @@
+/*
+ * Copyright (c) 2019 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *          http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.uber.hoodie.common.table.timeline.dto;
+
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.uber.hoodie.exception.HoodieException;
+import java.io.IOException;
+import org.apache.hadoop.fs.FileStatus;
+
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class FileStatusDTO {
+
+  @JsonProperty("path")
+  FilePathDTO path;
+  @JsonProperty("length")
+  long length;
+  @JsonProperty("isdir")
+  boolean isdir;
+  @JsonProperty("blockReplication")
+  short blockReplication;
+  @JsonProperty("blocksize")
+  long blocksize;
+  @JsonProperty("modificationTime")
+  long modificationTime;
+  @JsonProperty("accessTime")
+  long accessTime;
+  @JsonProperty("permission")
+  FSPermissionDTO permission;
+  @JsonProperty("owner")
+  String owner;
+  @JsonProperty("group")
+  String group;
+  @JsonProperty("symlink")
+  FilePathDTO symlink;
+
+  public static FileStatusDTO fromFileStatus(FileStatus fileStatus) {
+    if (null == fileStatus) {
+      return null;
+    }
+
+    FileStatusDTO dto = new FileStatusDTO();
+    try {
+      dto.path = FilePathDTO.fromPath(fileStatus.getPath());
+      dto.length = fileStatus.getLen();
+      dto.isdir = fileStatus.isDirectory();
+      dto.blockReplication = fileStatus.getReplication();
+      dto.blocksize = fileStatus.getBlockSize();
+      dto.modificationTime = fileStatus.getModificationTime();
+      dto.accessTime = fileStatus.getModificationTime();
+      dto.symlink = fileStatus.isSymlink() ? FilePathDTO.fromPath(fileStatus.getSymlink()) : null;
+      safeReadAndSetMetadata(dto, fileStatus);
+    } catch (IOException ioe) {
+      throw new HoodieException(ioe);
+    }
+    return dto;
+  }
+
+  /**
+   * Used to safely handle FileStatus calls which might fail on some FileSystem implementation.
+   * (DeprecatedLocalFileSystem)
+   */
+  private static void safeReadAndSetMetadata(FileStatusDTO dto, FileStatus fileStatus) {
+    try {
+      dto.owner = fileStatus.getOwner();
+      dto.group = fileStatus.getGroup();
+      dto.permission = FSPermissionDTO.fromFsPermission(fileStatus.getPermission());
+    } catch (IllegalArgumentException ie) {
+      // Deprecated File System (testing) does not work well with this call
+      // skipping
+    }
+  }
+
+  public static FileStatus toFileStatus(FileStatusDTO dto) {
+    if (null == dto) {
+      return null;
+    }
+
+    return new FileStatus(dto.length, dto.isdir, dto.blockReplication, dto.blocksize, dto.modificationTime,
+        dto.accessTime, FSPermissionDTO.fromFsPermissionDTO(dto.permission), dto.owner, dto.group,
+        FilePathDTO.toPath(dto.symlink), FilePathDTO.toPath(dto.path));
+  }
+}
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/dto/InstantDTO.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/dto/InstantDTO.java
new file mode 100644
index 0000000..ada5a6c
--- /dev/null
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/dto/InstantDTO.java
@@ -0,0 +1,53 @@
+/*
+ * Copyright (c) 2019 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *          http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.uber.hoodie.common.table.timeline.dto;
+
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.uber.hoodie.common.table.timeline.HoodieInstant;
+
+
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class InstantDTO {
+
+  @JsonProperty("action")
+  String action;
+  @JsonProperty("ts")
+  String timestamp;
+  @JsonProperty("state")
+  String state;
+
+  public static InstantDTO fromInstant(HoodieInstant instant) {
+    if (null == instant) {
+      return null;
+    }
+
+    InstantDTO dto = new InstantDTO();
+    dto.action = instant.getAction();
+    dto.timestamp = instant.getTimestamp();
+    dto.state = instant.getState().toString();
+    return dto;
+  }
+
+  public static HoodieInstant toInstant(InstantDTO dto) {
+    if (null == dto) {
+      return null;
+    }
+
+    return new HoodieInstant(HoodieInstant.State.valueOf(dto.state), dto.action, dto.timestamp);
+  }
+}
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/dto/LogFileDTO.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/dto/LogFileDTO.java
new file mode 100644
index 0000000..72cd5e9
--- /dev/null
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/dto/LogFileDTO.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright (c) 2019 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *          http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.uber.hoodie.common.table.timeline.dto;
+
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.uber.hoodie.common.model.HoodieLogFile;
+import org.apache.hadoop.fs.FileStatus;
+
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class LogFileDTO {
+
+  @JsonProperty("fileStatus")
+  private FileStatusDTO fileStatus;
+  @JsonProperty("path")
+  private String pathStr;
+  @JsonProperty("len")
+  private long fileLen;
+
+  public static HoodieLogFile toHoodieLogFile(LogFileDTO dto) {
+    FileStatus status = FileStatusDTO.toFileStatus(dto.fileStatus);
+    HoodieLogFile logFile = (status == null) ? new HoodieLogFile(dto.pathStr) : new HoodieLogFile(status);
+    logFile.setFileLen(dto.fileLen);
+    return logFile;
+  }
+
+  public static LogFileDTO fromHoodieLogFile(HoodieLogFile dataFile) {
+    LogFileDTO logFile = new LogFileDTO();
+    logFile.fileLen = dataFile.getFileSize();
+    logFile.pathStr = dataFile.getPath().toString();
+    logFile.fileStatus = FileStatusDTO.fromFileStatus(dataFile.getFileStatus());
+    return logFile;
+  }
+}
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/dto/TimelineDTO.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/dto/TimelineDTO.java
new file mode 100644
index 0000000..d3c6403
--- /dev/null
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/dto/TimelineDTO.java
@@ -0,0 +1,44 @@
+/*
+ * Copyright (c) 2019 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *          http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.uber.hoodie.common.table.timeline.dto;
+
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.uber.hoodie.common.table.HoodieTableMetaClient;
+import com.uber.hoodie.common.table.HoodieTimeline;
+import com.uber.hoodie.common.table.timeline.HoodieDefaultTimeline;
+import java.util.List;
+import java.util.stream.Collectors;
+
+@JsonIgnoreProperties(ignoreUnknown = true)
+public class TimelineDTO {
+
+  @JsonProperty("instants")
+  List<InstantDTO> instants;
+
+  public static TimelineDTO fromTimeline(HoodieTimeline timeline) {
+    TimelineDTO dto = new TimelineDTO();
+    dto.instants = timeline.getInstants().map(InstantDTO::fromInstant).collect(Collectors.toList());
+    return dto;
+  }
+
+  public static HoodieTimeline toTimeline(TimelineDTO dto, HoodieTableMetaClient metaClient) {
+    //TODO:  For Now, we will assume, only active-timeline will be transferred.
+    return new HoodieDefaultTimeline(dto.instants.stream().map(InstantDTO::toInstant),
+        metaClient.getActiveTimeline()::getInstantDetails);
+  }
+}
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/AbstractTableFileSystemView.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/AbstractTableFileSystemView.java
new file mode 100644
index 0000000..17f6b56
--- /dev/null
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/AbstractTableFileSystemView.java
@@ -0,0 +1,823 @@
+/*
+ *  Copyright (c) 2019 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  You may obtain a copy of the License at
+ *
+ *           http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package com.uber.hoodie.common.table.view;
+
+import com.google.common.base.Preconditions;
+import com.uber.hoodie.common.model.CompactionOperation;
+import com.uber.hoodie.common.model.FileSlice;
+import com.uber.hoodie.common.model.HoodieDataFile;
+import com.uber.hoodie.common.model.HoodieFileGroup;
+import com.uber.hoodie.common.model.HoodieFileGroupId;
+import com.uber.hoodie.common.model.HoodieLogFile;
+import com.uber.hoodie.common.table.HoodieTableMetaClient;
+import com.uber.hoodie.common.table.HoodieTimeline;
+import com.uber.hoodie.common.table.SyncableFileSystemView;
+import com.uber.hoodie.common.table.timeline.HoodieInstant;
+import com.uber.hoodie.common.util.CompactionUtils;
+import com.uber.hoodie.common.util.FSUtils;
+import com.uber.hoodie.common.util.HoodieTimer;
+import com.uber.hoodie.common.util.Option;
+import com.uber.hoodie.common.util.collection.Pair;
+import com.uber.hoodie.exception.HoodieIOException;
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+/**
+ * Common thread-safe implementation for multiple TableFileSystemView Implementations.
+ * Provides uniform handling of
+ *   (a) Loading file-system views from underlying file-system
+ *   (b) Pending compaction operations and changing file-system views based on that
+ *   (c) Thread-safety in loading and managing file system views for this dataset.
+ *   (d) resetting file-system views
+ * The actual mechanism of fetching file slices from different view storages is delegated to sub-classes.
+ */
+public abstract class AbstractTableFileSystemView implements SyncableFileSystemView, Serializable {
+
+  private static Logger log = LogManager.getLogger(AbstractTableFileSystemView.class);
+
+  protected HoodieTableMetaClient metaClient;
+
+  // This is the commits that will be visible for all views extending this view
+  protected HoodieTimeline visibleActiveTimeline;
+
+  // Used to concurrently load and populate partition views
+  private ConcurrentHashMap<String, Boolean> addedPartitions = new ConcurrentHashMap<>(4096);
+
+  // Locks to control concurrency. Sync operations use write-lock blocking all fetch operations.
+  // For the common-case, we allow concurrent read of single or multiple partitions
+  private final ReentrantReadWriteLock globalLock = new ReentrantReadWriteLock();
+  private final ReadLock readLock = globalLock.readLock();
+  private final WriteLock writeLock = globalLock.writeLock();
+
+  private String getPartitionPathFromFilePath(String fullPath) {
+    return FSUtils.getRelativePartitionPath(new Path(metaClient.getBasePath()), new Path(fullPath).getParent());
+  }
+
+  /**
+   * Initialize the view.
+   */
+  protected void init(HoodieTableMetaClient metaClient, HoodieTimeline visibleActiveTimeline) {
+    this.metaClient = metaClient;
+    this.visibleActiveTimeline = visibleActiveTimeline;
+    // Load Pending Compaction Operations
+    resetPendingCompactionOperations(
+        CompactionUtils.getAllPendingCompactionOperations(metaClient).values()
+            .stream().map(e -> Pair.of(e.getKey(),
+            CompactionOperation.convertFromAvroRecordInstance(e.getValue()))));
+  }
+
+  /**
+   * Adds the provided statuses into the file system view, and also caches it inside this object.
+   */
+  protected List<HoodieFileGroup> addFilesToView(FileStatus[] statuses) {
+    HoodieTimer timer = new HoodieTimer().startTimer();
+    List<HoodieFileGroup> fileGroups = buildFileGroups(statuses, visibleActiveTimeline, true);
+    long fgBuildTimeTakenMs = timer.endTimer();
+    timer.startTimer();
+    // Group by partition for efficient updates for both InMemory and DiskBased stuctures.
+    fileGroups.stream().collect(Collectors.groupingBy(HoodieFileGroup::getPartitionPath)).entrySet()
+        .forEach(entry -> {
+          String partition = entry.getKey();
+          if (!isPartitionAvailableInStore(partition)) {
+            storePartitionView(partition, entry.getValue());
+          }
+        });
+    long storePartitionsTs = timer.endTimer();
+    log.info("addFilesToView: NumFiles=" + statuses.length + ", FileGroupsCreationTime=" + fgBuildTimeTakenMs
+        + ", StoreTimeTaken=" + storePartitionsTs);
+    return fileGroups;
+  }
+
+  /**
+   * Build FileGroups from passed in file-status
+   */
+  protected List<HoodieFileGroup> buildFileGroups(FileStatus[] statuses, HoodieTimeline timeline,
+      boolean addPendingCompactionFileSlice) {
+    return buildFileGroups(convertFileStatusesToDataFiles(statuses), convertFileStatusesToLogFiles(statuses), timeline,
+        addPendingCompactionFileSlice);
+  }
+
+  protected List<HoodieFileGroup> buildFileGroups(Stream<HoodieDataFile> dataFileStream,
+      Stream<HoodieLogFile> logFileStream, HoodieTimeline timeline, boolean addPendingCompactionFileSlice) {
+
+    Map<Pair<String, String>, List<HoodieDataFile>> dataFiles = dataFileStream
+        .collect(Collectors.groupingBy((dataFile) -> {
+          String partitionPathStr = getPartitionPathFromFilePath(dataFile.getPath());
+          return Pair.of(partitionPathStr, dataFile.getFileId());
+        }));
+
+    Map<Pair<String, String>, List<HoodieLogFile>> logFiles = logFileStream
+        .collect(Collectors.groupingBy((logFile) -> {
+          String partitionPathStr = FSUtils.getRelativePartitionPath(
+              new Path(metaClient.getBasePath()),
+              logFile.getPath().getParent());
+          return Pair.of(partitionPathStr, logFile.getFileId());
+        }));
+
+    Set<Pair<String, String>> fileIdSet = new HashSet<>(dataFiles.keySet());
+    fileIdSet.addAll(logFiles.keySet());
+
+    List<HoodieFileGroup> fileGroups = new ArrayList<>();
+    fileIdSet.forEach(pair -> {
+      String fileId = pair.getValue();
+      HoodieFileGroup group = new HoodieFileGroup(pair.getKey(), fileId, timeline);
+      if (dataFiles.containsKey(pair)) {
+        dataFiles.get(pair).forEach(group::addDataFile);
+      }
+      if (logFiles.containsKey(pair)) {
+        logFiles.get(pair).forEach(group::addLogFile);
+      }
+      if (addPendingCompactionFileSlice) {
+        Option<Pair<String, CompactionOperation>> pendingCompaction =
+            getPendingCompactionOperationWithInstant(group.getFileGroupId());
+        if (pendingCompaction.isPresent()) {
+          // If there is no delta-commit after compaction request, this step would ensure a new file-slice appears
+          // so that any new ingestion uses the correct base-instant
+          group.addNewFileSliceAtInstant(pendingCompaction.get().getKey());
+        }
+      }
+      fileGroups.add(group);
+    });
+
+    return fileGroups;
+  }
+
+  /**
+   * Clears the partition Map and reset view states
+   */
+  public final void reset() {
+    try {
+      writeLock.lock();
+
+      addedPartitions.clear();
+      resetViewState();
+
+      // Initialize with new Hoodie timeline.
+      init(metaClient, visibleActiveTimeline);
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  /**
+   * Allows all view metadata in file system view storage to be reset by subclasses
+   */
+  protected abstract void resetViewState();
+
+  /**
+   * Allows lazily loading the partitions if needed
+   *
+   * @param partition partition to be loaded if not present
+   */
+  private void ensurePartitionLoadedCorrectly(String partition) {
+
+    Preconditions.checkArgument(!isClosed(), "View is already closed");
+
+    // ensure we list files only once even in the face of concurrency
+    addedPartitions.computeIfAbsent(partition, (partitionPathStr) -> {
+      long beginTs = System.currentTimeMillis();
+      if (!isPartitionAvailableInStore(partitionPathStr)) {
+        // Not loaded yet
+        try {
+          log.info("Building file system view for partition (" + partitionPathStr + ")");
+
+          // Create the path if it does not exist already
+          Path partitionPath = new Path(metaClient.getBasePath(), partitionPathStr);
+          FSUtils.createPathIfNotExists(metaClient.getFs(), partitionPath);
+          long beginLsTs = System.currentTimeMillis();
+          FileStatus[] statuses = metaClient.getFs().listStatus(partitionPath);
+          long endLsTs = System.currentTimeMillis();
+          log.info("#files found in partition (" + partitionPathStr + ") =" + statuses.length
+              + ", Time taken =" + (endLsTs - beginLsTs));
+          List<HoodieFileGroup> groups = addFilesToView(statuses);
+
+          if (groups.isEmpty()) {
+            storePartitionView(partitionPathStr, new ArrayList<>());
+          }
+        } catch (IOException e) {
+          throw new HoodieIOException("Failed to list data files in partition " + partitionPathStr, e);
+        }
+      } else {
+        log.debug("View already built for Partition :" + partitionPathStr + ", FOUND is ");
+      }
+      long endTs = System.currentTimeMillis();
+      log.info("Time to load partition (" + partitionPathStr + ") =" + (endTs - beginTs));
+      return true;
+    });
+  }
+
+  /**
+   * Helper to convert file-status to data-files
+   *
+   * @param statuses List of Fole-Status
+   */
+  private Stream<HoodieDataFile> convertFileStatusesToDataFiles(FileStatus[] statuses) {
+    Predicate<FileStatus> roFilePredicate = fileStatus ->
+        fileStatus.getPath().getName()
+            .contains(metaClient.getTableConfig().getROFileFormat().getFileExtension());
+    return Arrays.stream(statuses).filter(roFilePredicate).map(HoodieDataFile::new);
+  }
+
+  /**
+   * Helper to convert file-status to log-files
+   *
+   * @param statuses List of FIle-Status
+   */
+  private Stream<HoodieLogFile> convertFileStatusesToLogFiles(FileStatus[] statuses) {
+    Predicate<FileStatus> rtFilePredicate = fileStatus ->
+        fileStatus.getPath().getName()
+            .contains(metaClient.getTableConfig().getRTFileFormat().getFileExtension());
+    return Arrays.stream(statuses).filter(rtFilePredicate).map(HoodieLogFile::new);
+  }
+
+  /**
+   * With async compaction, it is possible to see partial/complete data-files due to inflight-compactions, Ignore those
+   * data-files
+   *
+   * @param dataFile Data File
+   */
+  protected boolean isDataFileDueToPendingCompaction(HoodieDataFile dataFile) {
+    final String partitionPath = getPartitionPathFromFilePath(dataFile.getPath());
+
+    Option<Pair<String, CompactionOperation>> compactionWithInstantTime =
+        getPendingCompactionOperationWithInstant(new HoodieFileGroupId(partitionPath, dataFile.getFileId()));
+    return (compactionWithInstantTime.isPresent()) && (null != compactionWithInstantTime.get().getKey())
+        && dataFile.getCommitTime().equals(compactionWithInstantTime.get().getKey());
+  }
+
+  /**
+   * Returns true if the file-group is under pending-compaction and the file-slice' baseInstant matches compaction
+   * Instant
+   *
+   * @param fileSlice File Slice
+   */
+  protected boolean isFileSliceAfterPendingCompaction(FileSlice fileSlice) {
+    Option<Pair<String, CompactionOperation>> compactionWithInstantTime =
+        getPendingCompactionOperationWithInstant(fileSlice.getFileGroupId());
+    return (compactionWithInstantTime.isPresent())
+        && fileSlice.getBaseInstantTime().equals(compactionWithInstantTime.get().getKey());
+  }
+
+  /**
+   * With async compaction, it is possible to see partial/complete data-files due to inflight-compactions, Ignore those
+   * data-files
+   *
+   * @param fileSlice File Slice
+   */
+  protected FileSlice filterDataFileAfterPendingCompaction(FileSlice fileSlice) {
+    if (isFileSliceAfterPendingCompaction(fileSlice)) {
+      // Data file is filtered out of the file-slice as the corresponding compaction
+      // instant not completed yet.
+      FileSlice transformed = new FileSlice(fileSlice.getPartitionPath(),
+          fileSlice.getBaseInstantTime(), fileSlice.getFileId());
+      fileSlice.getLogFiles().forEach(transformed::addLogFile);
+      return transformed;
+    }
+    return fileSlice;
+  }
+
+  @Override
+  public final Stream<Pair<String, CompactionOperation>> getPendingCompactionOperations() {
+    try {
+      readLock.lock();
+      return fetchPendingCompactionOperations();
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  @Override
+  public final Stream<HoodieDataFile> getLatestDataFiles(String partitionStr) {
+    try {
+      readLock.lock();
+      String partitionPath = formatPartitionKey(partitionStr);
+      ensurePartitionLoadedCorrectly(partitionPath);
+      return fetchLatestDataFiles(partitionPath);
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  @Override
+  public final Stream<HoodieDataFile> getLatestDataFiles() {
+    try {
+      readLock.lock();
+      return fetchLatestDataFiles();
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  @Override
+  public final Stream<HoodieDataFile> getLatestDataFilesBeforeOrOn(String partitionStr, String maxCommitTime) {
+    try {
+      readLock.lock();
+      String partitionPath = formatPartitionKey(partitionStr);
+      ensurePartitionLoadedCorrectly(partitionPath);
+      return fetchAllStoredFileGroups(partitionPath)
+          .map(fileGroup -> fileGroup.getAllDataFiles()
+              .filter(dataFile ->
+                  HoodieTimeline.compareTimestamps(dataFile.getCommitTime(),
+                      maxCommitTime,
+                      HoodieTimeline.LESSER_OR_EQUAL))
+              .filter(df -> !isDataFileDueToPendingCompaction(df))
+              .findFirst())
+          .filter(Optional::isPresent)
+          .map(Optional::get);
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  @Override
+  public final Option<HoodieDataFile> getDataFileOn(String partitionStr, String instantTime, String fileId) {
+    try {
+      readLock.lock();
+      String partitionPath = formatPartitionKey(partitionStr);
+      ensurePartitionLoadedCorrectly(partitionPath);
+      return fetchHoodieFileGroup(partitionPath, fileId)
+          .map(fileGroup -> fileGroup.getAllDataFiles()
+              .filter(dataFile ->
+                  HoodieTimeline.compareTimestamps(dataFile.getCommitTime(),
+                      instantTime, HoodieTimeline.EQUAL))
+              .filter(df -> !isDataFileDueToPendingCompaction(df))
+              .findFirst().orElse(null));
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  /**
+   * Get Latest data file for a partition and file-Id
+   */
+  public final Option<HoodieDataFile> getLatestDataFile(String partitionStr, String fileId) {
+    try {
+      readLock.lock();
+      String partitionPath = formatPartitionKey(partitionStr);
+      ensurePartitionLoadedCorrectly(partitionPath);
+      return fetchLatestDataFile(partitionPath, fileId);
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  @Override
+  public final Stream<HoodieDataFile> getLatestDataFilesInRange(List<String> commitsToReturn) {
+    try {
+      readLock.lock();
+      return fetchAllStoredFileGroups().map(fileGroup -> {
+        return fileGroup.getAllDataFiles()
+            .filter(dataFile -> commitsToReturn.contains(dataFile.getCommitTime())
+                && !isDataFileDueToPendingCompaction(dataFile))
+            .findFirst();
+      }).filter(Optional::isPresent).map(Optional::get);
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  @Override
+  public final Stream<HoodieDataFile> getAllDataFiles(String partitionStr) {
+    try {
+      readLock.lock();
+      String partitionPath = formatPartitionKey(partitionStr);
+      ensurePartitionLoadedCorrectly(partitionPath);
+      return fetchAllDataFiles(partitionPath)
+          .filter(df -> visibleActiveTimeline.containsOrBeforeTimelineStarts(df.getCommitTime()))
+          .filter(df -> !isDataFileDueToPendingCompaction(df));
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  @Override
+  public final Stream<FileSlice> getLatestFileSlices(String partitionStr) {
+    try {
+      readLock.lock();
+      String partitionPath = formatPartitionKey(partitionStr);
+      ensurePartitionLoadedCorrectly(partitionPath);
+      return fetchLatestFileSlices(partitionPath).map(fs -> filterDataFileAfterPendingCompaction(fs));
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  /**
+   * Get Latest File Slice for a given fileId in a given partition
+   */
+  public final Option<FileSlice> getLatestFileSlice(String partitionStr, String fileId) {
+    try {
+      readLock.lock();
+      String partitionPath = formatPartitionKey(partitionStr);
+      ensurePartitionLoadedCorrectly(partitionPath);
+      Option<FileSlice> fs = fetchLatestFileSlice(partitionPath, fileId);
+      return fs.map(f -> filterDataFileAfterPendingCompaction(f));
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  @Override
+  public final Stream<FileSlice> getLatestUnCompactedFileSlices(String partitionStr) {
+    try {
+      readLock.lock();
+      String partitionPath = formatPartitionKey(partitionStr);
+      ensurePartitionLoadedCorrectly(partitionPath);
+      return fetchAllStoredFileGroups(partitionPath)
+          .map(fileGroup -> {
+            FileSlice fileSlice = fileGroup.getLatestFileSlice().get();
+            // if the file-group is under compaction, pick the latest before compaction instant time.
+            Option<Pair<String, CompactionOperation>> compactionWithInstantPair =
+                getPendingCompactionOperationWithInstant(fileSlice.getFileGroupId());
+            if (compactionWithInstantPair.isPresent()) {
+              String compactionInstantTime = compactionWithInstantPair.get().getLeft();
+              return fileGroup.getLatestFileSliceBefore(compactionInstantTime);
+            }
+            return Optional.of(fileSlice);
+          })
+          .map(Optional::get);
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  @Override
+  public final Stream<FileSlice> getLatestFileSlicesBeforeOrOn(String partitionStr, String maxCommitTime) {
+    try {
+      readLock.lock();
+      String partitionPath = formatPartitionKey(partitionStr);
+      ensurePartitionLoadedCorrectly(partitionPath);
+      return fetchLatestFileSlicesBeforeOrOn(partitionPath, maxCommitTime)
+          .map(fs -> filterDataFileAfterPendingCompaction(fs));
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  @Override
+  public final Stream<FileSlice> getLatestMergedFileSlicesBeforeOrOn(String partitionStr, String maxInstantTime) {
+    try {
+      readLock.lock();
+      String partition = formatPartitionKey(partitionStr);
+      ensurePartitionLoadedCorrectly(partition);
+      return fetchAllStoredFileGroups(partition)
+          .map(fileGroup -> {
+            Optional<FileSlice> fileSlice = fileGroup.getLatestFileSliceBeforeOrOn(maxInstantTime);
+            // if the file-group is under construction, pick the latest before compaction instant time.
+            if (fileSlice.isPresent()) {
+              fileSlice = Optional.of(fetchMergedFileSlice(fileGroup, fileSlice.get()));
+            }
+            return fileSlice;
+          })
+          .filter(Optional::isPresent)
+          .map(Optional::get);
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  @Override
+  public final Stream<FileSlice> getLatestFileSliceInRange(List<String> commitsToReturn) {
+    try {
+      readLock.lock();
+      return fetchLatestFileSliceInRange(commitsToReturn);
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  @Override
+  public final Stream<FileSlice> getAllFileSlices(String partitionStr) {
+    try {
+      readLock.lock();
+      String partition = formatPartitionKey(partitionStr);
+      ensurePartitionLoadedCorrectly(partition);
+      return fetchAllFileSlices(partition);
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  /**
+   * Ensure there is consistency in handling trailing slash in partition-path. Always trim it which is what is done in
+   * other places.
+   */
+  private String formatPartitionKey(String partitionStr) {
+    return partitionStr.endsWith("/") ? partitionStr.substring(0, partitionStr.length() - 1) : partitionStr;
+  }
+
+  @Override
+  public final Stream<HoodieFileGroup> getAllFileGroups(String partitionStr) {
+    try {
+      readLock.lock();
+      // Ensure there is consistency in handling trailing slash in partition-path. Always trim it which is what is done
+      // in other places.
+      String partition = formatPartitionKey(partitionStr);
+      ensurePartitionLoadedCorrectly(partition);
+      return fetchAllStoredFileGroups(partition);
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  // Fetch APIs to be implemented by concrete sub-classes
+
+  /**
+   * Check if there is an outstanding compaction scheduled for this file
+   *
+   * @param fgId File-Group Id
+   * @return true if there is a pending compaction, false otherwise
+   */
+  protected abstract boolean isPendingCompactionScheduledForFileId(HoodieFileGroupId fgId);
+
+  /**
+   * resets the pending compaction operation and overwrite with the new list
+   *
+   * @param operations Pending Compaction Operations
+   */
+  abstract void resetPendingCompactionOperations(Stream<Pair<String, CompactionOperation>> operations);
+
+  /**
+   * Add pending compaction operations to store
+   *
+   * @param operations Pending compaction operations to be added
+   */
+  abstract void addPendingCompactionOperations(Stream<Pair<String, CompactionOperation>> operations);
+
+  /**
+   * Remove pending compaction operations from store
+   *
+   * @param operations Pending compaction operations to be removed
+   */
+  abstract void removePendingCompactionOperations(Stream<Pair<String, CompactionOperation>> operations);
+
+  /**
+   * Return pending compaction operation for a file-group
+   *
+   * @param fileGroupId File-Group Id
+   */
+  protected abstract Option<Pair<String, CompactionOperation>> getPendingCompactionOperationWithInstant(
+      HoodieFileGroupId fileGroupId);
+
+  /**
+   * Fetch all pending compaction operations
+   */
+  abstract Stream<Pair<String, CompactionOperation>> fetchPendingCompactionOperations();
+
+  /**
+   * Checks if partition is pre-loaded and available in store
+   *
+   * @param partitionPath Partition Path
+   */
+  abstract boolean isPartitionAvailableInStore(String partitionPath);
+
+  /**
+   * Add a complete partition view to store
+   *
+   * @param partitionPath Partition Path
+   * @param fileGroups File Groups for the partition path
+   */
+  abstract void storePartitionView(String partitionPath, List<HoodieFileGroup> fileGroups);
+
+  /**
+   * Fetch all file-groups stored for a partition-path
+   *
+   * @param partitionPath Partition path for which the file-groups needs to be retrieved.
+   * @return file-group stream
+   */
+  abstract Stream<HoodieFileGroup> fetchAllStoredFileGroups(String partitionPath);
+
+  /**
+   * Fetch all Stored file-groups across all partitions loaded
+   *
+   * @return file-group stream
+   */
+  abstract Stream<HoodieFileGroup> fetchAllStoredFileGroups();
+
+  /**
+   * Check if the view is already closed
+   */
+  abstract boolean isClosed();
+
+  /**
+   * Default implementation for fetching latest file-slice in commit range
+   *
+   * @param commitsToReturn Commits
+   */
+  Stream<FileSlice> fetchLatestFileSliceInRange(List<String> commitsToReturn) {
+    return fetchAllStoredFileGroups().map(fileGroup -> fileGroup.getLatestFileSliceInRange(commitsToReturn))
+        .map(Optional::get);
+  }
+
+  /**
+   * Default implementation for fetching all file-slices for a partition-path
+   *
+   * @param partitionPath Partition path
+   * @return file-slice stream
+   */
+  Stream<FileSlice> fetchAllFileSlices(String partitionPath) {
+    return fetchAllStoredFileGroups(partitionPath)
+        .map(HoodieFileGroup::getAllFileSlices)
+        .flatMap(sliceList -> sliceList);
+  }
+
+  /**
+   * Default implementation for fetching latest data-files for the partition-path
+   */
+  Stream<HoodieDataFile> fetchLatestDataFiles(final String partitionPath) {
+    return fetchAllStoredFileGroups(partitionPath)
+        .map(this::getLatestDataFile)
+        .filter(Optional::isPresent)
+        .map(Optional::get);
+  }
+
+
+  protected Optional<HoodieDataFile> getLatestDataFile(HoodieFileGroup fileGroup) {
+    return fileGroup.getAllDataFiles().filter(df -> !isDataFileDueToPendingCompaction(df)).findFirst();
+  }
+
+  /**
+   * Default implementation for fetching latest data-files across all partitions
+   */
+  Stream<HoodieDataFile> fetchLatestDataFiles() {
+    return fetchAllStoredFileGroups()
+        .map(this::getLatestDataFile)
+        .filter(Optional::isPresent)
+        .map(Optional::get);
+  }
+
+  /**
+   * Default implementation for fetching all data-files for a partition
+   *
+   * @param partitionPath partition-path
+   */
+  Stream<HoodieDataFile> fetchAllDataFiles(String partitionPath) {
+    return fetchAllStoredFileGroups(partitionPath)
+        .map(HoodieFileGroup::getAllDataFiles)
+        .flatMap(dataFileList -> dataFileList);
+  }
+
+  /**
+   * Default implementation for fetching file-group
+   */
+  Option<HoodieFileGroup> fetchHoodieFileGroup(String partitionPath, String fileId) {
+    return Option.fromJavaOptional(fetchAllStoredFileGroups(partitionPath)
+        .filter(fileGroup -> fileGroup.getFileGroupId().getFileId().equals(fileId)).findFirst());
+  }
+
+  /**
+   * Default implementation for fetching latest file-slices for a partition path
+   */
+  Stream<FileSlice> fetchLatestFileSlices(String partitionPath) {
+    return fetchAllStoredFileGroups(partitionPath)
+        .map(HoodieFileGroup::getLatestFileSlice)
+        .filter(Optional::isPresent)
+        .map(Optional::get);
+  }
+
+  /**
+   * Default implementation for fetching latest file-slices for a partition path as of instant
+   *
+   * @param partitionPath Partition Path
+   * @param maxCommitTime Instant Time
+   */
+  Stream<FileSlice> fetchLatestFileSlicesBeforeOrOn(String partitionPath,
+      String maxCommitTime) {
+    return fetchAllStoredFileGroups(partitionPath)
+        .map(fileGroup -> fileGroup.getLatestFileSliceBeforeOrOn(maxCommitTime))
+        .filter(Optional::isPresent)
+        .map(Optional::get);
+  }
+
+  /**
+   * Helper to merge last 2 file-slices. These 2 file-slices do not have compaction done yet.
+   *
+   * @param lastSlice Latest File slice for a file-group
+   * @param penultimateSlice Penultimate file slice for a file-group in commit timeline order
+   */
+  private static FileSlice mergeCompactionPendingFileSlices(FileSlice lastSlice, FileSlice penultimateSlice) {
+    FileSlice merged = new FileSlice(penultimateSlice.getPartitionPath(),
+        penultimateSlice.getBaseInstantTime(), penultimateSlice.getFileId());
+    if (penultimateSlice.getDataFile().isPresent()) {
+      merged.setDataFile(penultimateSlice.getDataFile().get());
+    }
+    // Add Log files from penultimate and last slices
+    penultimateSlice.getLogFiles().forEach(merged::addLogFile);
+    lastSlice.getLogFiles().forEach(merged::addLogFile);
+    return merged;
+  }
+
+  /**
+   * If the file-slice is because of pending compaction instant, this method merges the file-slice with the one before
+   * the compaction instant time
+   *
+   * @param fileGroup File Group for which the file slice belongs to
+   * @param fileSlice File Slice which needs to be merged
+   */
+  private FileSlice fetchMergedFileSlice(HoodieFileGroup fileGroup, FileSlice fileSlice) {
+    // if the file-group is under construction, pick the latest before compaction instant time.
+    Option<Pair<String, CompactionOperation>> compactionOpWithInstant =
+        getPendingCompactionOperationWithInstant(fileGroup.getFileGroupId());
+    if (compactionOpWithInstant.isPresent()) {
+      String compactionInstantTime = compactionOpWithInstant.get().getKey();
+      if (fileSlice.getBaseInstantTime().equals(compactionInstantTime)) {
+        Optional<FileSlice> prevFileSlice = fileGroup.getLatestFileSliceBefore(compactionInstantTime);
+        if (prevFileSlice.isPresent()) {
+          return mergeCompactionPendingFileSlices(fileSlice, prevFileSlice.get());
+        }
+      }
+    }
+    return fileSlice;
+  }
+
+  /**
+   * Default implementation for fetching latest data-file
+   * @param partitionPath Partition path
+   * @param fileId File Id
+   * @return Data File if present
+   */
+  protected Option<HoodieDataFile> fetchLatestDataFile(String partitionPath, String fileId) {
+    return Option.fromJavaOptional(fetchLatestDataFiles(partitionPath)
+        .filter(fs -> fs.getFileId().equals(fileId)).findFirst());
+  }
+
+  /**
+   * Default implementation for fetching file-slice
+   * @param partitionPath Partition path
+   * @param fileId File Id
+   * @return File Slice if present
+   */
+  protected Option<FileSlice> fetchLatestFileSlice(String partitionPath, String fileId) {
+    return Option.fromJavaOptional(fetchLatestFileSlices(partitionPath)
+        .filter(fs -> fs.getFileId().equals(fileId)).findFirst());
+  }
+
+  @Override
+  public Option<HoodieInstant> getLastInstant() {
+    return Option.fromJavaOptional(visibleActiveTimeline.lastInstant());
+  }
+
+  @Override
+  public HoodieTimeline getTimeline() {
+    return visibleActiveTimeline;
+  }
+
+  @Override
+  public void sync() {
+    HoodieTimeline oldTimeline = getTimeline();
+    HoodieTimeline newTimeline = metaClient.reloadActiveTimeline().filterCompletedAndCompactionInstants();
+    try {
+      writeLock.lock();
+      runSync(oldTimeline, newTimeline);
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  /**
+   * Performs complete reset of file-system view. Subsequent partition view calls will load file slices against latest
+   * timeline
+   *
+   * @param oldTimeline Old Hoodie Timeline
+   * @param newTimeline New Hoodie Timeline
+   */
+  protected void runSync(HoodieTimeline oldTimeline, HoodieTimeline newTimeline) {
+    visibleActiveTimeline = newTimeline;
+    addedPartitions.clear();
+    resetViewState();
+    // Initialize with new Hoodie timeline.
+    init(metaClient, newTimeline);
+  }
+}
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/FileSystemViewManager.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/FileSystemViewManager.java
new file mode 100644
index 0000000..c3853f6
--- /dev/null
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/FileSystemViewManager.java
@@ -0,0 +1,216 @@
+/*
+ * Copyright (c) 2019 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *          http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.uber.hoodie.common.table.view;
+
+import com.uber.hoodie.common.SerializableConfiguration;
+import com.uber.hoodie.common.table.HoodieTableMetaClient;
+import com.uber.hoodie.common.table.HoodieTimeline;
+import com.uber.hoodie.common.table.SyncableFileSystemView;
+import com.uber.hoodie.common.util.Functions.Function2;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+/**
+ * A container that can potentially hold one or more dataset's
+ * file-system views. There is one view for each dataset. This is a view built against a timeline containing completed
+ * actions. In an embedded timeline-server mode, this typically holds only one dataset's view.
+ * In a stand-alone server mode, this can hold more than one dataset's views.
+ *
+ * FileSystemView can be stored "locally" using the following storage mechanisms:
+ *  a. In Memory
+ *  b. Spillable Map
+ *  c. RocksDB
+ *
+ * But there can be cases where the file-system view is managed remoted. For example : Embedded Timeline Server). In
+ * this case, the clients will configure a remote filesystem view client (RemoteHoodieTableFileSystemView) for the
+ * dataset which can connect to the remote file system view and fetch views. THere are 2 modes here : REMOTE_FIRST and
+ * REMOTE_ONLY
+ *    REMOTE_FIRST : The file-system view implementation on client side will act as a remote proxy. In case, if there
+ *                   is problem (or exceptions) querying remote file-system view, a backup local file-system view(using
+ *                   either one of in-memory, spillable, rocksDB) is used to server file-system view queries
+ *    REMOTE_ONLY  : In this case, there is no backup local file-system view. If there is problem (or exceptions)
+ *                   querying remote file-system view, then the exceptions are percolated back to client.
+ *
+ * FileSystemViewManager is designed to encapsulate the file-system view storage from clients using the file-system
+ * view. FileSystemViewManager uses a factory to construct specific implementation of file-system view and passes it to
+ * clients for querying.
+ */
+public class FileSystemViewManager {
+  private static Logger logger = LogManager.getLogger(FileSystemViewManager.class);
+
+  private final SerializableConfiguration conf;
+  // The View Storage config used to store file-system views
+  private final FileSystemViewStorageConfig viewStorageConfig;
+  // Map from Base-Path to View
+  private final ConcurrentHashMap<String, SyncableFileSystemView> globalViewMap;
+  // Factory Map to create file-system views
+  private final Function2<String, FileSystemViewStorageConfig, SyncableFileSystemView> viewCreator;
+
+  public FileSystemViewManager(SerializableConfiguration conf, FileSystemViewStorageConfig viewStorageConfig,
+      Function2<String, FileSystemViewStorageConfig, SyncableFileSystemView> viewCreator) {
+    this.conf = conf;
+    this.viewStorageConfig = viewStorageConfig;
+    this.globalViewMap = new ConcurrentHashMap<>();
+    this.viewCreator = viewCreator;
+  }
+
+  /**
+   * Drops reference to File-System Views. Future calls to view results in creating a new view
+   * @param basePath
+   */
+  public void clearFileSystemView(String basePath) {
+    SyncableFileSystemView view = globalViewMap.remove(basePath);
+    if (view != null) {
+      view.close();
+    }
+  }
+
+  /**
+   * Main API to get the file-system view for the base-path
+   * @param basePath
+   * @return
+   */
+  public SyncableFileSystemView getFileSystemView(String basePath) {
+    return globalViewMap.computeIfAbsent(basePath,
+        (path) -> viewCreator.apply(path, viewStorageConfig));
+  }
+
+  /**
+   * Closes all views opened
+   */
+  public void close() {
+    this.globalViewMap.values().stream().forEach(v -> v.close());
+    this.globalViewMap.clear();
+  }
+
+  // FACTORY METHODS FOR CREATING FILE-SYSTEM VIEWS
+
+  /**
+   * Create RocksDB based file System view for a dataset
+   * @param conf Hadoop Configuration
+   * @param viewConf  View Storage Configuration
+   * @param basePath  Base Path of dataset
+   * @return
+   */
+  private static RocksDbBasedFileSystemView createRocksDBBasedFileSystemView(SerializableConfiguration conf,
+      FileSystemViewStorageConfig viewConf, String basePath) {
+    HoodieTableMetaClient metaClient = new HoodieTableMetaClient(conf.get(), basePath, true);
+    HoodieTimeline timeline = metaClient.getActiveTimeline().filterCompletedAndCompactionInstants();
+    return new RocksDbBasedFileSystemView(metaClient, timeline, viewConf);
+  }
+
+  /**
+   * Create a spillable Map based file System view for a dataset
+   * @param conf Hadoop Configuration
+   * @param viewConf  View Storage Configuration
+   * @param basePath  Base Path of dataset
+   * @return
+   */
+  private static SpillableMapBasedFileSystemView createSpillableMapBasedFileSystemView(SerializableConfiguration conf,
+      FileSystemViewStorageConfig viewConf, String basePath) {
+    logger.info("Creating SpillableMap based view for basePath " + basePath);
+    HoodieTableMetaClient metaClient = new HoodieTableMetaClient(conf.get(), basePath, true);
+    HoodieTimeline timeline = metaClient.getActiveTimeline().filterCompletedAndCompactionInstants();
+    return new SpillableMapBasedFileSystemView(metaClient, timeline, viewConf);
+  }
+
+
+  /**
+   * Create an in-memory file System view for a dataset
+   * @param conf Hadoop Configuration
+   * @param viewConf  View Storage Configuration
+   * @param basePath  Base Path of dataset
+   * @return
+   */
+  private static HoodieTableFileSystemView createInMemoryFileSystemView(SerializableConfiguration conf,
+      FileSystemViewStorageConfig viewConf, String basePath) {
+    logger.info("Creating InMemory based view for basePath " + basePath);
+    HoodieTableMetaClient metaClient = new HoodieTableMetaClient(conf.get(), basePath, true);
+    HoodieTimeline timeline = metaClient.getActiveTimeline().filterCompletedAndCompactionInstants();
+    return new HoodieTableFileSystemView(metaClient, timeline, viewConf.isIncrementalTimelineSyncEnabled());
+  }
+
+  /**
+   * Create a remote file System view for a dataset
+   * @param conf Hadoop Configuration
+   * @param viewConf  View Storage Configuration
+   * @param metaClient  Hoodie Table MetaClient for the dataset.
+   * @return
+   */
+  private static RemoteHoodieTableFileSystemView createRemoteFileSystemView(SerializableConfiguration conf,
+      FileSystemViewStorageConfig viewConf, HoodieTableMetaClient metaClient) {
+    logger.info("Creating remote view for basePath " + metaClient.getBasePath() + ". Server="
+        + viewConf.getRemoteViewServerHost() + ":" + viewConf.getRemoteViewServerPort());
+    return new RemoteHoodieTableFileSystemView(viewConf.getRemoteViewServerHost(),
+        viewConf.getRemoteViewServerPort(), metaClient);
+  }
+
+  /**
+   * Main Factory method for building file-system views
+   * @param conf  Hadoop Configuration
+   * @param config View Storage Configuration
+   * @return
+   */
+  public static FileSystemViewManager createViewManager(
+      final SerializableConfiguration conf, final FileSystemViewStorageConfig config) {
+    logger.info("Creating View Manager with storage type :" + config.getStorageType());
+    switch (config.getStorageType()) {
+      case EMBEDDED_KV_STORE:
+        logger.info("Creating embedded rocks-db based Table View");
+        return new FileSystemViewManager(conf, config,
+            (basePath, viewConf) -> createRocksDBBasedFileSystemView(conf, viewConf, basePath));
+      case SPILLABLE_DISK:
+        logger.info("Creating Spillable Disk based Table View");
+        return new FileSystemViewManager(conf, config,
+            (basePath, viewConf) -> createSpillableMapBasedFileSystemView(conf, viewConf, basePath));
+      case MEMORY:
+        logger.info("Creating in-memory based Table View");
+        return new FileSystemViewManager(conf, config,
+            (basePath, viewConfig) -> createInMemoryFileSystemView(conf, viewConfig, basePath));
+      case REMOTE_ONLY:
+        logger.info("Creating remote only table view");
+        return new FileSystemViewManager(conf, config,
+            (basePath, viewConfig) -> createRemoteFileSystemView(conf, viewConfig,
+                new HoodieTableMetaClient(conf.get(), basePath)));
+      case REMOTE_FIRST:
+        logger.info("Creating remote first table view");
+        return new FileSystemViewManager(conf, config, (basePath, viewConfig) -> {
+          RemoteHoodieTableFileSystemView remoteFileSystemView =
+              createRemoteFileSystemView(conf, viewConfig, new HoodieTableMetaClient(conf.get(), basePath));
+          SyncableFileSystemView secondaryView = null;
+          switch (viewConfig.getSecondaryStorageType()) {
+            case MEMORY:
+              secondaryView = createInMemoryFileSystemView(conf, viewConfig, basePath);
+              break;
+            case EMBEDDED_KV_STORE:
+              secondaryView = createRocksDBBasedFileSystemView(conf, viewConfig, basePath);
+              break;
+            case SPILLABLE_DISK:
+              secondaryView = createSpillableMapBasedFileSystemView(conf, viewConfig, basePath);
+              break;
+            default:
+              throw new IllegalArgumentException("Secondary Storage type can only be in-memory or spillable. Was :"
+                  + viewConfig.getSecondaryStorageType());
+          }
+          return new PriorityBasedFileSystemView(remoteFileSystemView, secondaryView);
+        });
+      default:
+        throw new IllegalArgumentException("Unknown file system view type :" + config.getStorageType());
+    }
+  }
+}
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/FileSystemViewStorageConfig.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/FileSystemViewStorageConfig.java
new file mode 100644
index 0000000..97742d9
--- /dev/null
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/FileSystemViewStorageConfig.java
@@ -0,0 +1,197 @@
+/*
+ * Copyright (c) 2019 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *          http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.uber.hoodie.common.table.view;
+
+import com.google.common.base.Preconditions;
+import com.uber.hoodie.config.DefaultHoodieConfig;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.Properties;
+
+/**
+ * File System View Storage Configurations
+ */
+public class FileSystemViewStorageConfig extends DefaultHoodieConfig {
+
+  //Property Names
+  public static final String FILESYSTEM_VIEW_STORAGE_TYPE = "hoodie.filesystem.view.type";
+  public static final String FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE = "hoodie.filesystem.view.incr.timeline.sync.enable";
+  public static final String FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE = "hoodie.filesystem.view.secondary.type";
+  public static final String FILESYSTEM_VIEW_REMOTE_HOST = "hoodie.filesystem.view.remote.host";
+  public static final String FILESYSTEM_VIEW_REMOTE_PORT = "hoodie.filesystem.view.remote.port";
+  public static final String FILESYSTEM_VIEW_SPILLABLE_DIR = "hoodie.filesystem.view.spillable.dir";
+  public static final String FILESYSTEM_VIEW_SPILLABLE_MEM = "hoodie.filesystem.view.spillable.mem";
+  public static final String FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION =
+      "hoodie.filesystem.view.spillable.compaction.mem.fraction";
+  private static final String ROCKSDB_BASE_PATH_PROP = "hoodie.filesystem.view.rocksdb.base.path";
+
+  public static final FileSystemViewStorageType DEFAULT_VIEW_STORAGE_TYPE = FileSystemViewStorageType.MEMORY;
+  public static final FileSystemViewStorageType DEFAULT_SECONDARY_VIEW_STORAGE_TYPE = FileSystemViewStorageType.MEMORY;
+  public static final String DEFAULT_ROCKSDB_BASE_PATH = "/tmp/hoodie_timeline_rocksdb";
+
+  public static final String DEFAULT_FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE = "false";
+  public static final String DEFUALT_REMOTE_VIEW_SERVER_HOST = "localhost";
+  public static final Integer DEFAULT_REMOTE_VIEW_SERVER_PORT = 26754;
+
+  public static final String DEFAULT_VIEW_SPILLABLE_DIR = "/tmp/view_map/";
+  private static final Double DEFAULT_MEM_FRACTION_FOR_PENDING_COMPACTION = 0.01;
+  private static final Long DEFAULT_MAX_MEMORY_FOR_VIEW = 100 * 1024 * 1024L; // 100 MB
+
+  public static FileSystemViewStorageConfig.Builder newBuilder() {
+    return new Builder();
+  }
+
+  private FileSystemViewStorageConfig(Properties props) {
+    super(props);
+  }
+
+  public FileSystemViewStorageType getStorageType() {
+    return FileSystemViewStorageType.valueOf(props.getProperty(FILESYSTEM_VIEW_STORAGE_TYPE));
+  }
+
+  public boolean isIncrementalTimelineSyncEnabled() {
+    return Boolean.valueOf(props.getProperty(FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE));
+  }
+
+  public String getRemoteViewServerHost() {
+    return props.getProperty(FILESYSTEM_VIEW_REMOTE_HOST);
+  }
+
+  public Integer getRemoteViewServerPort() {
+    return Integer.parseInt(props.getProperty(FILESYSTEM_VIEW_REMOTE_PORT));
+  }
+
+  public long getMaxMemoryForFileGroupMap() {
+    long totalMemory = Long.parseLong(props.getProperty(FILESYSTEM_VIEW_SPILLABLE_MEM));
+    return totalMemory - getMaxMemoryForPendingCompaction();
+  }
+
+  public long getMaxMemoryForPendingCompaction() {
+    long totalMemory = Long.parseLong(props.getProperty(FILESYSTEM_VIEW_SPILLABLE_MEM));
+    long reservedForPendingComaction = new Double(totalMemory * Double.parseDouble(
+        props.getProperty(FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION))).longValue();
+    return reservedForPendingComaction;
+  }
+
+  public String getBaseStoreDir() {
+    return props.getProperty(FILESYSTEM_VIEW_SPILLABLE_DIR);
+  }
+
+  public FileSystemViewStorageType getSecondaryStorageType() {
+    return FileSystemViewStorageType.valueOf(props.getProperty(FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE));
+  }
+
+  public String getRocksdbBasePath() {
+    return props.getProperty(ROCKSDB_BASE_PATH_PROP);
+  }
+
+  public static class Builder {
+
+    private final Properties props = new Properties();
+
+    public Builder fromFile(File propertiesFile) throws IOException {
+      FileReader reader = new FileReader(propertiesFile);
+      try {
+        props.load(reader);
+        return this;
+      } finally {
+        reader.close();
+      }
+    }
+
+    public Builder fromProperties(Properties props) {
+      this.props.putAll(props);
+      return this;
+    }
+
+    public Builder withStorageType(FileSystemViewStorageType storageType) {
+      props.setProperty(FILESYSTEM_VIEW_STORAGE_TYPE, storageType.name());
+      return this;
+    }
+
+    public Builder withSecondaryStorageType(FileSystemViewStorageType storageType) {
+      props.setProperty(FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE, storageType.name());
+      return this;
+    }
+
+    public Builder withIncrementalTimelineSync(boolean enableIncrTimelineSync) {
+      props.setProperty(FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE, Boolean.toString(enableIncrTimelineSync));
+      return this;
+    }
+
+    public Builder withRemoteServerHost(String remoteServerHost) {
+      props.setProperty(FILESYSTEM_VIEW_REMOTE_HOST, remoteServerHost);
+      return this;
+    }
+
+    public Builder withRemoteServerPort(Integer remoteServerPort) {
+      props.setProperty(FILESYSTEM_VIEW_REMOTE_PORT, remoteServerPort.toString());
+      return this;
+    }
+
+    public Builder withMaxMemoryForView(Long maxMemoryForView) {
+      props.setProperty(FILESYSTEM_VIEW_SPILLABLE_MEM, maxMemoryForView.toString());
+      return this;
+    }
+
+    public Builder withMemFractionForPendingCompaction(Double memFractionForPendingCompaction) {
+      props.setProperty(FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION, memFractionForPendingCompaction.toString());
+      return this;
+    }
+
+    public Builder withBaseStoreDir(String baseStorePath) {
+      props.setProperty(FILESYSTEM_VIEW_SPILLABLE_DIR, baseStorePath);
+      return this;
+    }
+
+    public Builder withRocksDBPath(String basePath) {
+      props.setProperty(ROCKSDB_BASE_PATH_PROP, basePath);
+      return this;
+    }
+
+    public FileSystemViewStorageConfig build() {
+      setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_STORAGE_TYPE),
+          FILESYSTEM_VIEW_STORAGE_TYPE, DEFAULT_VIEW_STORAGE_TYPE.name());
+      setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE),
+          FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE, DEFAULT_FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE);
+      setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE),
+          FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE, DEFAULT_SECONDARY_VIEW_STORAGE_TYPE.name());
+      setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_REMOTE_HOST),
+          FILESYSTEM_VIEW_REMOTE_HOST, DEFUALT_REMOTE_VIEW_SERVER_HOST);
+      setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_REMOTE_PORT),
+          FILESYSTEM_VIEW_REMOTE_PORT, DEFAULT_REMOTE_VIEW_SERVER_PORT.toString());
+
+      setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_SPILLABLE_DIR),
+          FILESYSTEM_VIEW_SPILLABLE_DIR, DEFAULT_VIEW_SPILLABLE_DIR);
+      setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_SPILLABLE_MEM),
+          FILESYSTEM_VIEW_SPILLABLE_MEM, DEFAULT_MAX_MEMORY_FOR_VIEW.toString());
+      setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION),
+          FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION, DEFAULT_MEM_FRACTION_FOR_PENDING_COMPACTION.toString());
+
+      setDefaultOnCondition(props, !props.containsKey(ROCKSDB_BASE_PATH_PROP),
+          ROCKSDB_BASE_PATH_PROP, DEFAULT_ROCKSDB_BASE_PATH);
+
+      // Validations
+      FileSystemViewStorageType.valueOf(props.getProperty(FILESYSTEM_VIEW_STORAGE_TYPE));
+      FileSystemViewStorageType.valueOf(props.getProperty(FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE));
+      Preconditions.checkArgument(Integer.parseInt(props.getProperty(FILESYSTEM_VIEW_REMOTE_PORT)) > 0);
+      return new FileSystemViewStorageConfig(props);
+    }
+  }
+
+}
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/FileSystemViewStorageType.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/FileSystemViewStorageType.java
new file mode 100644
index 0000000..437792f
--- /dev/null
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/FileSystemViewStorageType.java
@@ -0,0 +1,34 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *          http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.uber.hoodie.common.table.view;
+
+/**
+ * Storage Type used to store/retrieve File system view of a table
+ */
+public enum FileSystemViewStorageType {
+  // In-memory storage of file-system view
+  MEMORY,
+  // Constrained Memory storage for file-system view with overflow data spilled to disk
+  SPILLABLE_DISK,
+  // EMBEDDED Key Value Storage for file-system view
+  EMBEDDED_KV_STORE,
+  // Delegate file-system view to remote server
+  REMOTE_ONLY,
+  // A composite storage where file-system view calls are first delegated to Remote server ( REMOTE_ONLY )
+  // In case of failures, switches subsequent calls to secondary local storage type
+  REMOTE_FIRST
+}
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java
index 93f61b7..2f555dd 100644
--- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemView.java
@@ -16,432 +16,202 @@
 
 package com.uber.hoodie.common.table.view;
 
-import com.google.common.collect.ImmutableMap;
+import com.google.common.base.Preconditions;
 import com.uber.hoodie.common.model.CompactionOperation;
-import com.uber.hoodie.common.model.FileSlice;
-import com.uber.hoodie.common.model.HoodieDataFile;
 import com.uber.hoodie.common.model.HoodieFileGroup;
 import com.uber.hoodie.common.model.HoodieFileGroupId;
-import com.uber.hoodie.common.model.HoodieLogFile;
 import com.uber.hoodie.common.table.HoodieTableMetaClient;
 import com.uber.hoodie.common.table.HoodieTimeline;
 import com.uber.hoodie.common.table.TableFileSystemView;
-import com.uber.hoodie.common.table.timeline.HoodieInstant;
-import com.uber.hoodie.common.util.CompactionUtils;
-import com.uber.hoodie.common.util.FSUtils;
+import com.uber.hoodie.common.util.Option;
 import com.uber.hoodie.common.util.collection.Pair;
-import com.uber.hoodie.exception.HoodieIOException;
 import java.io.IOException;
-import java.io.Serializable;
 import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
-import java.util.Optional;
-import java.util.Set;
-import java.util.function.Predicate;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.Path;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
 
 /**
- * Common abstract implementation for multiple TableFileSystemView Implementations. 2 possible
- * implementations are ReadOptimizedView and RealtimeView <p> Concrete implementations extending
- * this abstract class, should only implement getDataFilesInPartition which includes files to be
- * included in the view
- *
+ * TableFileSystemView Implementations based on in-memory storage.
  * @see TableFileSystemView
  * @since 0.3.0
  */
-public class HoodieTableFileSystemView implements TableFileSystemView,
-    TableFileSystemView.ReadOptimizedView,
-    TableFileSystemView.RealtimeView, Serializable {
+public class HoodieTableFileSystemView extends IncrementalTimelineSyncFileSystemView {
 
   private static Logger log = LogManager.getLogger(HoodieTableFileSystemView.class);
 
-  protected HoodieTableMetaClient metaClient;
-  // This is the commits that will be visible for all views extending this view
-  protected HoodieTimeline visibleActiveTimeline;
-
   // mapping from partition paths to file groups contained within them
-  protected HashMap<String, List<HoodieFileGroup>> partitionToFileGroupsMap;
-  // mapping from file id to the file group.
-  protected HashMap<HoodieFileGroupId, HoodieFileGroup> fileGroupMap;
+  protected Map<String, List<HoodieFileGroup>> partitionToFileGroupsMap;
 
   /**
    * PartitionPath + File-Id to pending compaction instant time
    */
-  private final Map<HoodieFileGroupId, Pair<String, CompactionOperation>> fgIdToPendingCompaction;
+  protected Map<HoodieFileGroupId, Pair<String, CompactionOperation>> fgIdToPendingCompaction;
 
   /**
-   * Create a file system view, as of the given timeline
+   * Flag to determine if closed
    */
-  public HoodieTableFileSystemView(HoodieTableMetaClient metaClient,
-      HoodieTimeline visibleActiveTimeline) {
-    this.metaClient = metaClient;
-    this.visibleActiveTimeline = visibleActiveTimeline;
-    this.fileGroupMap = new HashMap<>();
-    this.partitionToFileGroupsMap = new HashMap<>();
+  private boolean closed = false;
 
-    // Build fileId to Pending Compaction Instants
-    List<HoodieInstant> pendingCompactionInstants =
-        metaClient.getActiveTimeline().filterPendingCompactionTimeline().getInstants().collect(Collectors.toList());
-    this.fgIdToPendingCompaction = ImmutableMap.copyOf(
-        CompactionUtils.getAllPendingCompactionOperations(metaClient).entrySet().stream()
-                .map(entry -> Pair.of(entry.getKey(), Pair.of(entry.getValue().getKey(),
-                        CompactionOperation.convertFromAvroRecordInstance(entry.getValue().getValue()))))
-                .collect(Collectors.toMap(Pair::getKey, Pair::getValue)));
+  HoodieTableFileSystemView(boolean enableIncrementalTimelineSync) {
+    super(enableIncrementalTimelineSync);
   }
 
   /**
-   * Create a file system view, as of the given timeline, with the provided file statuses.
+   * Create a file system view, as of the given timeline
    */
-  public HoodieTableFileSystemView(HoodieTableMetaClient metaClient,
-      HoodieTimeline visibleActiveTimeline,
-      FileStatus[] fileStatuses) {
-    this(metaClient, visibleActiveTimeline);
-    addFilesToView(fileStatuses);
+  public HoodieTableFileSystemView(HoodieTableMetaClient metaClient, HoodieTimeline visibleActiveTimeline) {
+    this(metaClient, visibleActiveTimeline, false);
   }
 
-
   /**
-   * This method is only used when this object is deserialized in a spark executor.
-   *
-   * @deprecated
+   * Create a file system view, as of the given timeline
    */
-  private void readObject(java.io.ObjectInputStream in)
-      throws IOException, ClassNotFoundException {
-    in.defaultReadObject();
+  public HoodieTableFileSystemView(HoodieTableMetaClient metaClient, HoodieTimeline visibleActiveTimeline,
+      boolean enableIncrementalTimelineSync) {
+    super(enableIncrementalTimelineSync);
+    init(metaClient, visibleActiveTimeline);
   }
 
-  private void writeObject(java.io.ObjectOutputStream out)
-      throws IOException {
-    out.defaultWriteObject();
+  @Override
+  public void init(HoodieTableMetaClient metaClient, HoodieTimeline visibleActiveTimeline) {
+    this.partitionToFileGroupsMap = createPartitionToFileGroups();
+    super.init(metaClient, visibleActiveTimeline);
   }
 
-  private String getPartitionPathFromFileStatus(FileStatus fileStatus) {
-    return FSUtils.getRelativePartitionPath(new Path(metaClient.getBasePath()), fileStatus.getPath().getParent());
+  @Override
+  protected void resetViewState() {
+    this.fgIdToPendingCompaction = null;
+    this.partitionToFileGroupsMap = null;
   }
 
-  /**
-   * Adds the provided statuses into the file system view, and also caches it inside this object.
-   */
-  private List<HoodieFileGroup> addFilesToView(FileStatus[] statuses) {
-    Map<Pair<String, String>, List<HoodieDataFile>> dataFiles = convertFileStatusesToDataFiles(
-        statuses)
-        .collect(Collectors.groupingBy((dataFile) -> {
-          String partitionPathStr = getPartitionPathFromFileStatus(dataFile.getFileStatus());
-          return Pair.of(partitionPathStr, dataFile.getFileId());
-        }));
-    Map<Pair<String, String>, List<HoodieLogFile>> logFiles = convertFileStatusesToLogFiles(
-        statuses)
-        .collect(Collectors.groupingBy((logFile) -> {
-          String partitionPathStr = FSUtils.getRelativePartitionPath(
-              new Path(metaClient.getBasePath()),
-              logFile.getPath().getParent());
-          return Pair.of(partitionPathStr, logFile.getFileId());
-        }));
-
-    Set<Pair<String, String>> fileIdSet = new HashSet<>(dataFiles.keySet());
-    fileIdSet.addAll(logFiles.keySet());
-
-    List<HoodieFileGroup> fileGroups = new ArrayList<>();
-    fileIdSet.forEach(pair -> {
-      String fileId = pair.getValue();
-      HoodieFileGroup group = new HoodieFileGroup(pair.getKey(), fileId, visibleActiveTimeline);
-      if (dataFiles.containsKey(pair)) {
-        dataFiles.get(pair).forEach(group::addDataFile);
-      }
-      if (logFiles.containsKey(pair)) {
-        logFiles.get(pair).forEach(group::addLogFile);
-      }
-      HoodieFileGroupId fgId = group.getFileGroupId();
-      if (fgIdToPendingCompaction.containsKey(fgId)) {
-        // If there is no delta-commit after compaction request, this step would ensure a new file-slice appears
-        // so that any new ingestion uses the correct base-instant
-        group.addNewFileSliceAtInstant(fgIdToPendingCompaction.get(fgId).getKey());
-      }
-      fileGroups.add(group);
-    });
-
-    // add to the cache.
-    fileGroups.forEach(group -> {
-      fileGroupMap.put(group.getFileGroupId(), group);
-      if (!partitionToFileGroupsMap.containsKey(group.getPartitionPath())) {
-        partitionToFileGroupsMap.put(group.getPartitionPath(), new ArrayList<>());
-      }
-      partitionToFileGroupsMap.get(group.getPartitionPath()).add(group);
-    });
-
-    return fileGroups;
+  protected Map<String, List<HoodieFileGroup>> createPartitionToFileGroups() {
+    return new ConcurrentHashMap<>();
   }
 
-  private Stream<HoodieDataFile> convertFileStatusesToDataFiles(FileStatus[] statuses) {
-    Predicate<FileStatus> roFilePredicate = fileStatus ->
-        fileStatus.getPath().getName()
-            .contains(metaClient.getTableConfig().getROFileFormat().getFileExtension());
-    return Arrays.stream(statuses).filter(roFilePredicate).map(HoodieDataFile::new);
+  protected Map<HoodieFileGroupId, Pair<String, CompactionOperation>> createFileIdToPendingCompactionMap(
+      Map<HoodieFileGroupId, Pair<String, CompactionOperation>> fileIdToPendingCompaction) {
+    return fileIdToPendingCompaction;
   }
 
-  private Stream<HoodieLogFile> convertFileStatusesToLogFiles(FileStatus[] statuses) {
-    Predicate<FileStatus> rtFilePredicate = fileStatus ->
-        fileStatus.getPath().getName()
-            .contains(metaClient.getTableConfig().getRTFileFormat().getFileExtension());
-    return Arrays.stream(statuses).filter(rtFilePredicate).map(HoodieLogFile::new);
+  /**
+   * Create a file system view, as of the given timeline, with the provided file statuses.
+   */
+  public HoodieTableFileSystemView(HoodieTableMetaClient metaClient, HoodieTimeline visibleActiveTimeline,
+      FileStatus[] fileStatuses) {
+    this(metaClient, visibleActiveTimeline);
+    addFilesToView(fileStatuses);
   }
 
   /**
-   * With async compaction, it is possible to see partial/complete data-files due to inflight-compactions, Ignore
-   * those data-files
+   * This method is only used when this object is deserialized in a spark executor.
    *
-   * @param dataFile Data File
+   * @deprecated
    */
-  private boolean isDataFileDueToPendingCompaction(HoodieDataFile dataFile) {
-    final String partitionPath = getPartitionPathFromFileStatus(dataFile.getFileStatus());
-    HoodieFileGroupId fgId = new HoodieFileGroupId(partitionPath, dataFile.getFileId());
-    Pair<String, CompactionOperation> compactionWithInstantTime = fgIdToPendingCompaction.get(fgId);
-    if ((null != compactionWithInstantTime) && (null != compactionWithInstantTime.getLeft())
-        && dataFile.getCommitTime().equals(compactionWithInstantTime.getKey())) {
-      return true;
-    }
-    return false;
-  }
-
-  @Override
-  public Stream<HoodieDataFile> getLatestDataFiles(final String partitionPath) {
-    return getAllFileGroups(partitionPath)
-        .map(fileGroup -> {
-          return fileGroup.getAllDataFiles()
-              .filter(df -> !isDataFileDueToPendingCompaction(df)).findFirst();
-        })
-        .filter(Optional::isPresent)
-        .map(Optional::get);
-  }
-
-  @Override
-  public Stream<HoodieDataFile> getLatestDataFiles() {
-    return fileGroupMap.values().stream()
-        .map(fileGroup -> fileGroup.getAllDataFiles().filter(df -> !isDataFileDueToPendingCompaction(df)).findFirst())
-        .filter(Optional::isPresent)
-        .map(Optional::get);
-  }
-
-  @Override
-  public Stream<HoodieDataFile> getLatestDataFilesBeforeOrOn(String partitionPath,
-      String maxCommitTime) {
-    return getAllFileGroups(partitionPath)
-        .map(fileGroup -> fileGroup.getAllDataFiles()
-            .filter(dataFile ->
-                HoodieTimeline.compareTimestamps(dataFile.getCommitTime(),
-                    maxCommitTime,
-                    HoodieTimeline.LESSER_OR_EQUAL))
-            .filter(df -> !isDataFileDueToPendingCompaction(df))
-            .findFirst())
-        .filter(Optional::isPresent)
-        .map(Optional::get);
+  private void readObject(java.io.ObjectInputStream in)
+      throws IOException, ClassNotFoundException {
+    in.defaultReadObject();
   }
 
-  @Override
-  public Stream<HoodieDataFile> getLatestDataFilesInRange(List<String> commitsToReturn) {
-    return fileGroupMap.values().stream()
-        .map(fileGroup -> fileGroup.getAllDataFiles()
-            .filter(dataFile -> commitsToReturn.contains(dataFile.getCommitTime())
-                && !isDataFileDueToPendingCompaction(dataFile))
-            .findFirst())
-        .filter(Optional::isPresent)
-        .map(Optional::get);
+  private void writeObject(java.io.ObjectOutputStream out)
+      throws IOException {
+    out.defaultWriteObject();
   }
 
   @Override
-  public Stream<HoodieDataFile> getLatestDataFilesOn(String partitionPath, String instantTime) {
-    return getAllFileGroups(partitionPath)
-        .map(fileGroup -> fileGroup.getAllDataFiles()
-            .filter(dataFile ->
-                HoodieTimeline.compareTimestamps(dataFile.getCommitTime(),
-                    instantTime,
-                    HoodieTimeline.EQUAL))
-            .filter(df -> !isDataFileDueToPendingCompaction(df))
-            .findFirst())
-        .filter(Optional::isPresent)
-        .map(Optional::get);
+  protected boolean isPendingCompactionScheduledForFileId(HoodieFileGroupId fgId) {
+    return fgIdToPendingCompaction.containsKey(fgId);
   }
 
   @Override
-  public Stream<HoodieDataFile> getAllDataFiles(String partitionPath) {
-    return getAllFileGroups(partitionPath)
-        .map(HoodieFileGroup::getAllDataFiles)
-        .flatMap(dataFileList -> dataFileList)
-        .filter(df -> !isDataFileDueToPendingCompaction(df));
+  protected void resetPendingCompactionOperations(Stream<Pair<String, CompactionOperation>> operations) {
+    // Build fileId to Pending Compaction Instants
+    this.fgIdToPendingCompaction = createFileIdToPendingCompactionMap(
+        operations.map(entry -> {
+          return Pair.of(entry.getValue().getFileGroupId(), Pair.of(entry.getKey(),entry.getValue()));
+        }).collect(Collectors.toMap(Pair::getKey, Pair::getValue)));
   }
 
   @Override
-  public Stream<FileSlice> getLatestFileSlices(String partitionPath) {
-    return getAllFileGroups(partitionPath)
-        .map(HoodieFileGroup::getLatestFileSlice)
-        .filter(Optional::isPresent)
-        .map(Optional::get)
-        .map(fs -> filterDataFileAfterPendingCompaction(fs));
+  protected void addPendingCompactionOperations(Stream<Pair<String, CompactionOperation>> operations) {
+    operations.forEach(opInstantPair -> {
+      Preconditions.checkArgument(!fgIdToPendingCompaction.containsKey(opInstantPair.getValue().getFileGroupId()),
+          "Duplicate FileGroupId found in pending compaction operations. FgId :"
+              + opInstantPair.getValue().getFileGroupId());
+      fgIdToPendingCompaction.put(opInstantPair.getValue().getFileGroupId(),
+          Pair.of(opInstantPair.getKey(), opInstantPair.getValue()));
+    });
   }
 
   @Override
-  public Stream<FileSlice> getLatestUnCompactedFileSlices(String partitionPath) {
-    return getAllFileGroups(partitionPath)
-        .map(fileGroup -> {
-          FileSlice fileSlice = fileGroup.getLatestFileSlice().get();
-          // if the file-group is under compaction, pick the latest before compaction instant time.
-          if (isFileSliceAfterPendingCompaction(fileSlice)) {
-            String compactionInstantTime = fgIdToPendingCompaction.get(fileSlice.getFileGroupId()).getLeft();
-            return fileGroup.getLatestFileSliceBefore(compactionInstantTime);
-          }
-          return Optional.of(fileSlice);
-        })
-        .map(Optional::get);
+  protected void removePendingCompactionOperations(Stream<Pair<String, CompactionOperation>> operations) {
+    operations.forEach(opInstantPair -> {
+      Preconditions.checkArgument(fgIdToPendingCompaction.containsKey(opInstantPair.getValue().getFileGroupId()),
+          "Trying to remove a FileGroupId which is not found in pending compaction operations. FgId :"
+              + opInstantPair.getValue().getFileGroupId());
+      fgIdToPendingCompaction.remove(opInstantPair.getValue().getFileGroupId());
+    });
   }
 
   /**
-   * Returns true if the file-group is under pending-compaction and the file-slice' baseInstant matches
-   * compaction Instant
-   * @param fileSlice File Slice
-   * @return
+   * Given a partition path, obtain all filegroups within that. All methods, that work at the
+   * partition level go through this.
    */
-  private boolean isFileSliceAfterPendingCompaction(FileSlice fileSlice) {
-    Pair<String, CompactionOperation> compactionWithInstantTime =
-        fgIdToPendingCompaction.get(fileSlice.getFileGroupId());
-    return (null != compactionWithInstantTime)
-            && fileSlice.getBaseInstantTime().equals(compactionWithInstantTime.getKey());
+  @Override
+  Stream<HoodieFileGroup> fetchAllStoredFileGroups(String partition) {
+    final List<HoodieFileGroup> fileGroups = new ArrayList<>();
+    fileGroups.addAll(partitionToFileGroupsMap.get(partition));
+    return fileGroups.stream();
   }
 
-  /**
-   * With async compaction, it is possible to see partial/complete data-files due to inflight-compactions,
-   * Ignore those data-files
-   * @param fileSlice File Slice
-   * @return
-   */
-  private FileSlice filterDataFileAfterPendingCompaction(FileSlice fileSlice) {
-    if (isFileSliceAfterPendingCompaction(fileSlice)) {
-      // Data file is filtered out of the file-slice as the corresponding compaction
-      // instant not completed yet.
-      FileSlice transformed = new FileSlice(fileSlice.getPartitionPath(),
-          fileSlice.getBaseInstantTime(), fileSlice.getFileId());
-      fileSlice.getLogFiles().forEach(transformed::addLogFile);
-      return transformed;
-    }
-    return fileSlice;
+  public Stream<HoodieFileGroup> getAllFileGroups() {
+    return fetchAllStoredFileGroups();
   }
 
   @Override
-  public Stream<FileSlice> getLatestFileSlicesBeforeOrOn(String partitionPath,
-      String maxCommitTime) {
-    return getAllFileGroups(partitionPath)
-        .map(fileGroup -> fileGroup.getLatestFileSliceBeforeOrOn(maxCommitTime))
-        .filter(Optional::isPresent)
-        .map(Optional::get)
-        .map(fs -> filterDataFileAfterPendingCompaction(fs));
-  }
+  Stream<Pair<String, CompactionOperation>> fetchPendingCompactionOperations() {
+    return fgIdToPendingCompaction.values().stream();
 
-  /**
-   * Helper to merge last 2 file-slices. These 2 file-slices do not have compaction done yet.
-   *
-   * @param lastSlice        Latest File slice for a file-group
-   * @param penultimateSlice Penultimate file slice for a file-group in commit timeline order
-   */
-  private static FileSlice mergeCompactionPendingFileSlices(FileSlice lastSlice, FileSlice penultimateSlice) {
-    FileSlice merged = new FileSlice(penultimateSlice.getPartitionPath(),
-        penultimateSlice.getBaseInstantTime(), penultimateSlice.getFileId());
-    if (penultimateSlice.getDataFile().isPresent()) {
-      merged.setDataFile(penultimateSlice.getDataFile().get());
-    }
-    // Add Log files from penultimate and last slices
-    penultimateSlice.getLogFiles().forEach(merged::addLogFile);
-    lastSlice.getLogFiles().forEach(merged::addLogFile);
-    return merged;
-  }
-
-  /**
-   * If the file-slice is because of pending compaction instant, this method merges the file-slice with the one before
-   * the compaction instant time
-   * @param fileGroup File Group for which the file slice belongs to
-   * @param fileSlice File Slice which needs to be merged
-   * @return
-   */
-  private FileSlice getMergedFileSlice(HoodieFileGroup fileGroup, FileSlice fileSlice) {
-    // if the file-group is under construction, pick the latest before compaction instant time.
-    HoodieFileGroupId fgId = fileSlice.getFileGroupId();
-    if (fgIdToPendingCompaction.containsKey(fgId)) {
-      String compactionInstantTime = fgIdToPendingCompaction.get(fgId).getKey();
-      if (fileSlice.getBaseInstantTime().equals(compactionInstantTime)) {
-        Optional<FileSlice> prevFileSlice = fileGroup.getLatestFileSliceBefore(compactionInstantTime);
-        if (prevFileSlice.isPresent()) {
-          return mergeCompactionPendingFileSlices(fileSlice, prevFileSlice.get());
-        }
-      }
-    }
-    return fileSlice;
   }
 
   @Override
-  public Stream<FileSlice> getLatestMergedFileSlicesBeforeOrOn(String partitionPath, String maxInstantTime) {
-    return getAllFileGroups(partitionPath)
-        .map(fileGroup -> {
-          Optional<FileSlice> fileSlice = fileGroup.getLatestFileSliceBeforeOrOn(maxInstantTime);
-          // if the file-group is under construction, pick the latest before compaction instant time.
-          if (fileSlice.isPresent()) {
-            fileSlice = Optional.of(getMergedFileSlice(fileGroup, fileSlice.get()));
-          }
-          return fileSlice;
-        })
-        .filter(Optional::isPresent)
-        .map(Optional::get);
+  protected Option<Pair<String, CompactionOperation>> getPendingCompactionOperationWithInstant(HoodieFileGroupId fgId) {
+    return Option.ofNullable(fgIdToPendingCompaction.get(fgId));
   }
 
   @Override
-  public Stream<FileSlice> getLatestFileSliceInRange(List<String> commitsToReturn) {
-    return fileGroupMap.values().stream()
-        .map(fileGroup -> fileGroup.getLatestFileSliceInRange(commitsToReturn))
-        .map(Optional::get);
+  protected boolean isPartitionAvailableInStore(String partitionPath) {
+    return partitionToFileGroupsMap.containsKey(partitionPath);
   }
 
   @Override
-  public Stream<FileSlice> getAllFileSlices(String partitionPath) {
-    return getAllFileGroups(partitionPath)
-        .map(HoodieFileGroup::getAllFileSlices)
-        .flatMap(sliceList -> sliceList);
+  protected void storePartitionView(String partitionPath, List<HoodieFileGroup> fileGroups) {
+    log.info("Adding file-groups for partition :" + partitionPath + ", #FileGroups=" + fileGroups.size());
+    List<HoodieFileGroup> newList = new ArrayList<>(fileGroups);
+    partitionToFileGroupsMap.put(partitionPath, newList);
   }
 
-  /**
-   * Given a partition path, obtain all filegroups within that. All methods, that work at the
-   * partition level go through this.
-   */
+
   @Override
-  public Stream<HoodieFileGroup> getAllFileGroups(String partitionPathStr) {
-    // return any previously fetched groups.
-    if (partitionToFileGroupsMap.containsKey(partitionPathStr)) {
-      return partitionToFileGroupsMap.get(partitionPathStr).stream();
-    }
-
-    try {
-      // Create the path if it does not exist already
-      Path partitionPath = FSUtils.getPartitionPath(metaClient.getBasePath(), partitionPathStr);
-      FSUtils.createPathIfNotExists(metaClient.getFs(), partitionPath);
-      FileStatus[] statuses = metaClient.getFs().listStatus(partitionPath);
-      List<HoodieFileGroup> fileGroups = addFilesToView(statuses);
-      return fileGroups.stream();
-    } catch (IOException e) {
-      throw new HoodieIOException(
-          "Failed to list data files in partition " + partitionPathStr, e);
-    }
+  public Stream<HoodieFileGroup> fetchAllStoredFileGroups() {
+    return partitionToFileGroupsMap.values().stream().flatMap(fg -> {
+      return fg.stream();
+    });
   }
 
-  public Map<HoodieFileGroupId, Pair<String, CompactionOperation>> getFgIdToPendingCompaction() {
-    return fgIdToPendingCompaction;
+  public void close() {
+    closed = true;
+    super.reset();
+    partitionToFileGroupsMap = null;
+    fgIdToPendingCompaction = null;
   }
 
-  public Stream<HoodieFileGroup> getAllFileGroups() {
-    return fileGroupMap.values().stream();
+  public boolean isClosed() {
+    return closed;
   }
 }
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/IncrementalTimelineSyncFileSystemView.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/IncrementalTimelineSyncFileSystemView.java
new file mode 100644
index 0000000..b18519e
--- /dev/null
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/IncrementalTimelineSyncFileSystemView.java
@@ -0,0 +1,338 @@
+/*
+ *  Copyright (c) 2019 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  You may obtain a copy of the License at
+ *
+ *           http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package com.uber.hoodie.common.table.view;
+
+import com.uber.hoodie.avro.model.HoodieCleanMetadata;
+import com.uber.hoodie.avro.model.HoodieCompactionPlan;
+import com.uber.hoodie.avro.model.HoodieRestoreMetadata;
+import com.uber.hoodie.avro.model.HoodieRollbackMetadata;
+import com.uber.hoodie.common.model.CompactionOperation;
+import com.uber.hoodie.common.model.FileSlice;
+import com.uber.hoodie.common.model.HoodieCommitMetadata;
+import com.uber.hoodie.common.model.HoodieDataFile;
+import com.uber.hoodie.common.model.HoodieFileGroup;
+import com.uber.hoodie.common.model.HoodieLogFile;
+import com.uber.hoodie.common.table.HoodieTimeline;
+import com.uber.hoodie.common.table.timeline.HoodieInstant;
+import com.uber.hoodie.common.util.AvroUtils;
+import com.uber.hoodie.common.util.CompactionUtils;
+import com.uber.hoodie.common.util.TimelineDiffHelper;
+import com.uber.hoodie.common.util.TimelineDiffHelper.TimelineDiffResult;
+import com.uber.hoodie.common.util.collection.Pair;
+import com.uber.hoodie.exception.HoodieException;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+/**
+ * Adds the capability to incrementally sync the changes to file-system view as and when new instants gets completed.
+ */
+public abstract class IncrementalTimelineSyncFileSystemView extends AbstractTableFileSystemView {
+
+  private static Logger log = LogManager.getLogger(IncrementalTimelineSyncFileSystemView.class);
+
+  // Allows incremental Timeline syncing
+  private final boolean incrementalTimelineSyncEnabled;
+
+  protected IncrementalTimelineSyncFileSystemView(boolean enableIncrementalTimelineSync) {
+    this.incrementalTimelineSyncEnabled = enableIncrementalTimelineSync;
+  }
+
+  @Override
+  protected void runSync(HoodieTimeline oldTimeline, HoodieTimeline newTimeline) {
+    try {
+      if (incrementalTimelineSyncEnabled) {
+        TimelineDiffResult diffResult = TimelineDiffHelper.getNewInstantsForIncrementalSync(oldTimeline, newTimeline);
+        if (diffResult.canSyncIncrementally()) {
+          log.info("Doing incremental sync");
+          runIncrementalSync(newTimeline, diffResult);
+          log.info("Finished incremental sync");
+          // Reset timeline to latest
+          visibleActiveTimeline = newTimeline;
+          return;
+        }
+      }
+    } catch (Exception ioe) {
+      log.error("Got exception trying to perform incremental sync. Reverting to complete sync", ioe);
+    }
+
+    log.warn("Incremental Sync of timeline is turned off or deemed unsafe. Will revert to full syncing");
+    super.runSync(oldTimeline, newTimeline);
+  }
+
+  /**
+   * Run incremental sync based on the diff result produced.
+   *
+   * @param timeline New Timeline
+   * @param diffResult Timeline Diff Result
+   */
+  private void runIncrementalSync(HoodieTimeline timeline, TimelineDiffResult diffResult) {
+
+    log.info("Timeline Diff Result is :" + diffResult);
+
+    // First remove pending compaction instants which were completed
+    diffResult.getFinishedCompactionInstants().stream().forEach(instant -> {
+      try {
+        removePendingCompactionInstant(timeline, instant);
+      } catch (IOException e) {
+        throw new HoodieException(e);
+      }
+    });
+
+    // Add new completed instants found in the latest timeline
+    diffResult.getNewlySeenInstants().stream()
+        .filter(instant -> instant.isCompleted() || instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION))
+        .forEach(instant -> {
+          try {
+            if (instant.getAction().equals(HoodieTimeline.COMMIT_ACTION)
+                || instant.getAction().equals(HoodieTimeline.DELTA_COMMIT_ACTION)) {
+              addCommitInstant(timeline, instant);
+            } else if (instant.getAction().equals(HoodieTimeline.RESTORE_ACTION)) {
+              addRestoreInstant(timeline, instant);
+            } else if (instant.getAction().equals(HoodieTimeline.CLEAN_ACTION)) {
+              addCleanInstant(timeline, instant);
+            } else if (instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION)) {
+              addPendingCompactionInstant(timeline, instant);
+            } else if (instant.getAction().equals(HoodieTimeline.ROLLBACK_ACTION)) {
+              addRollbackInstant(timeline, instant);
+            }
+          } catch (IOException ioe) {
+            throw new HoodieException(ioe);
+          }
+        });
+  }
+
+  /**
+   * Remove Pending compaction instant
+   *
+   * @param timeline New Hoodie Timeline
+   * @param instant Compaction Instant to be removed
+   */
+  private void removePendingCompactionInstant(HoodieTimeline timeline, HoodieInstant instant) throws IOException {
+    log.info("Removing completed compaction instant (" + instant + ")");
+    HoodieCompactionPlan plan = CompactionUtils.getCompactionPlan(metaClient, instant.getTimestamp());
+    removePendingCompactionOperations(CompactionUtils.getPendingCompactionOperations(instant, plan)
+        .map(instantPair -> Pair.of(instantPair.getValue().getKey(),
+            CompactionOperation.convertFromAvroRecordInstance(instantPair.getValue().getValue()))));
+  }
+
+  /**
+   * Add newly found compaction instant
+   *
+   * @param timeline Hoodie Timeline
+   * @param instant Compaction Instant
+   */
+  private void addPendingCompactionInstant(HoodieTimeline timeline, HoodieInstant instant) throws IOException {
+    log.info("Syncing pending compaction instant (" + instant + ")");
+    HoodieCompactionPlan compactionPlan = CompactionUtils.getCompactionPlan(metaClient, instant.getTimestamp());
+    List<Pair<String, CompactionOperation>> pendingOps =
+        CompactionUtils.getPendingCompactionOperations(instant, compactionPlan).map(p -> Pair.of(p.getValue().getKey(),
+            CompactionOperation.convertFromAvroRecordInstance(p.getValue().getValue()))).collect(Collectors.toList());
+    // First, update Pending compaction instants
+    addPendingCompactionOperations(pendingOps.stream());
+
+    Map<String, List<Pair<String, HoodieFileGroup>>> partitionToFileGroups =
+        pendingOps.stream().map(opPair -> {
+          String compactionInstantTime = opPair.getKey();
+          HoodieFileGroup fileGroup = new HoodieFileGroup(opPair.getValue().getFileGroupId(), timeline);
+          fileGroup.addNewFileSliceAtInstant(compactionInstantTime);
+          return Pair.of(compactionInstantTime, fileGroup);
+        }).collect(Collectors.groupingBy(x -> x.getValue().getPartitionPath()));
+    partitionToFileGroups.entrySet().forEach(entry -> {
+      if (isPartitionAvailableInStore(entry.getKey())) {
+        applyDeltaFileSlicesToPartitionView(entry.getKey(),
+            entry.getValue().stream().map(Pair::getValue).collect(Collectors.toList()), DeltaApplyMode.ADD);
+      }
+    });
+  }
+
+  /**
+   * Add newly found commit/delta-commit instant
+   *
+   * @param timeline Hoodie Timeline
+   * @param instant Instant
+   */
+  private void addCommitInstant(HoodieTimeline timeline, HoodieInstant instant) throws IOException {
+    log.info("Syncing committed instant (" + instant + ")");
+    HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(instant).get(),
+        HoodieCommitMetadata.class);
+    commitMetadata.getPartitionToWriteStats().entrySet().stream().forEach(entry -> {
+      String partition = entry.getKey();
+      if (isPartitionAvailableInStore(partition)) {
+        log.info("Syncing partition (" + partition + ") of instant (" + instant + ")");
+        FileStatus[] statuses = entry.getValue().stream().map(p -> {
+          FileStatus status = new FileStatus(p.getFileSizeInBytes(), false, 0, 0, 0, 0, null, null, null,
+              new Path(String.format("%s/%s", metaClient.getBasePath(), p.getPath())));
+          return status;
+        }).toArray(FileStatus[]::new);
+        List<HoodieFileGroup> fileGroups = buildFileGroups(statuses, timeline.filterCompletedAndCompactionInstants(),
+            false);
+        applyDeltaFileSlicesToPartitionView(partition, fileGroups, DeltaApplyMode.ADD);
+      } else {
+        log.warn("Skipping partition (" + partition + ") when syncing instant (" + instant + ") as it is not loaded");
+      }
+    });
+    log.info("Done Syncing committed instant (" + instant + ")");
+  }
+
+  /**
+   * Add newly found restore instant
+   *
+   * @param timeline Hoodie Timeline
+   * @param instant Restore Instant
+   */
+  private void addRestoreInstant(HoodieTimeline timeline, HoodieInstant instant) throws IOException {
+    log.info("Syncing restore instant (" + instant + ")");
+    HoodieRestoreMetadata metadata = AvroUtils.deserializeAvroMetadata(
+        timeline.getInstantDetails(instant).get(), HoodieRestoreMetadata.class);
+
+    Map<String, List<Pair<String, String>>> partitionFiles =
+        metadata.getHoodieRestoreMetadata().entrySet().stream().flatMap(entry -> {
+          return entry.getValue().stream().flatMap(e -> e.getPartitionMetadata().entrySet().stream().flatMap(e2 -> {
+            return e2.getValue().getSuccessDeleteFiles().stream().map(x -> Pair.of(e2.getKey(), x));
+          }));
+        }).collect(Collectors.groupingBy(Pair::getKey));
+    partitionFiles.entrySet().stream().forEach(e -> {
+      removeFileSlicesForPartition(timeline, instant, e.getKey(),
+          e.getValue().stream().map(x -> x.getValue()).collect(Collectors.toList()));
+    });
+    log.info("Done Syncing restore instant (" + instant + ")");
+  }
+
+  /**
+   * Add newly found rollback instant
+   *
+   * @param timeline Hoodie Timeline
+   * @param instant Rollback Instant
+   */
+  private void addRollbackInstant(HoodieTimeline timeline, HoodieInstant instant) throws IOException {
+    log.info("Syncing rollback instant (" + instant + ")");
+    HoodieRollbackMetadata metadata = AvroUtils.deserializeAvroMetadata(
+        timeline.getInstantDetails(instant).get(), HoodieRollbackMetadata.class);
+
+    metadata.getPartitionMetadata().entrySet().stream().forEach(e -> {
+      removeFileSlicesForPartition(timeline, instant, e.getKey(), e.getValue().getSuccessDeleteFiles());
+    });
+    log.info("Done Syncing rollback instant (" + instant + ")");
+  }
+
+  /**
+   * Add newly found clean instant
+   *
+   * @param timeline Timeline
+   * @param instant Clean instant
+   */
+  private void addCleanInstant(HoodieTimeline timeline, HoodieInstant instant) throws IOException {
+    log.info("Syncing cleaner instant (" + instant + ")");
+    HoodieCleanMetadata cleanMetadata = AvroUtils
+        .deserializeHoodieCleanMetadata(timeline.getInstantDetails(instant).get());
+    cleanMetadata.getPartitionMetadata().entrySet().stream().forEach(entry -> {
+      removeFileSlicesForPartition(timeline, instant, entry.getKey(), entry.getValue().getSuccessDeleteFiles());
+    });
+    log.info("Done Syncing cleaner instant (" + instant + ")");
+  }
+
+  private void removeFileSlicesForPartition(HoodieTimeline timeline, HoodieInstant instant,
+      String partition, List<String> paths) {
+    if (isPartitionAvailableInStore(partition)) {
+      log.info("Removing file slices for partition (" + partition + ") for instant (" + instant + ")");
+      FileStatus[] statuses = paths.stream().map(p -> {
+        FileStatus status = new FileStatus();
+        status.setPath(new Path(p));
+        return status;
+      }).toArray(FileStatus[]::new);
+      List<HoodieFileGroup> fileGroups = buildFileGroups(statuses,
+          timeline.filterCompletedAndCompactionInstants(), false);
+      applyDeltaFileSlicesToPartitionView(partition, fileGroups, DeltaApplyMode.REMOVE);
+    } else {
+      log.warn("Skipping partition (" + partition + ") when syncing instant (" + instant + ") as it is not loaded");
+    }
+  }
+
+  /**
+   * Apply mode whether to add or remove the delta view
+   */
+  enum DeltaApplyMode {
+    ADD,
+    REMOVE
+  }
+
+  /**
+   * Apply changes to partition file-system view. Base Implementation overwrites the entire partitions view assuming
+   * some sort of map (in-mem/disk-based) is used. For View implementation which supports fine-granular updates (e:g
+   * RocksDB), override this method.
+   *
+   * @param partition PartitionPath
+   * @param deltaFileGroups Changed file-slices aggregated as file-groups
+   * @param mode Delta Apply mode
+   */
+  protected void applyDeltaFileSlicesToPartitionView(String partition, List<HoodieFileGroup> deltaFileGroups,
+      DeltaApplyMode mode) {
+    if (deltaFileGroups.isEmpty()) {
+      log.info("No delta file groups for partition :" + partition);
+      return;
+    }
+
+    List<HoodieFileGroup> fileGroups = fetchAllStoredFileGroups(partition).collect(Collectors.toList());
+    /**
+     * Note that while finding the new data/log files added/removed, the path stored in metadata will be missing
+     * the base-path,scheme and authority. Ensure the matching process takes care of this discrepancy.
+     */
+    Map<String, HoodieDataFile> viewDataFiles = fileGroups.stream().flatMap(HoodieFileGroup::getAllRawFileSlices)
+        .map(FileSlice::getDataFile).filter(Optional::isPresent).map(Optional::get)
+        .map(df -> Pair.of(Path.getPathWithoutSchemeAndAuthority(new Path(df.getPath())).toString(), df))
+        .collect(Collectors.toMap(Pair::getKey, Pair::getValue));
+    //Note: Delta Log Files and Data FIles can be empty when adding/removing pending compactions
+    Map<String, HoodieDataFile> deltaDataFiles = deltaFileGroups.stream().flatMap(HoodieFileGroup::getAllRawFileSlices)
+        .map(FileSlice::getDataFile).filter(Optional::isPresent).map(Optional::get)
+        .map(df -> Pair.of(Path.getPathWithoutSchemeAndAuthority(new Path(df.getPath())).toString(), df))
+        .collect(Collectors.toMap(Pair::getKey, Pair::getValue));
+
+    Map<String, HoodieLogFile> viewLogFiles = fileGroups.stream().flatMap(HoodieFileGroup::getAllRawFileSlices)
+        .flatMap(FileSlice::getLogFiles)
+        .map(lf -> Pair.of(Path.getPathWithoutSchemeAndAuthority(lf.getPath()).toString(), lf))
+        .collect(Collectors.toMap(Pair::getKey, Pair::getValue));
+    Map<String, HoodieLogFile> deltaLogFiles = deltaFileGroups.stream().flatMap(HoodieFileGroup::getAllRawFileSlices)
+        .flatMap(FileSlice::getLogFiles)
+        .map(lf -> Pair.of(Path.getPathWithoutSchemeAndAuthority(lf.getPath()).toString(), lf))
+        .collect(Collectors.toMap(Pair::getKey, Pair::getValue));
+
+    switch (mode) {
+      case ADD:
+        viewDataFiles.putAll(deltaDataFiles);
+        viewLogFiles.putAll(deltaLogFiles);
+        break;
+      case REMOVE:
+        deltaDataFiles.keySet().stream().forEach(p -> viewDataFiles.remove(p));
+        deltaLogFiles.keySet().stream().forEach(p -> viewLogFiles.remove(p));
+        break;
+      default:
+        throw new IllegalStateException("Unknown diff apply mode=" + mode);
+    }
+
+    HoodieTimeline timeline = deltaFileGroups.stream().map(df -> df.getTimeline()).findAny().get();
+    List<HoodieFileGroup> fgs =
+        buildFileGroups(viewDataFiles.values().stream(), viewLogFiles.values().stream(), timeline, true);
+    storePartitionView(partition, fgs);
+  }
+}
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/PriorityBasedFileSystemView.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/PriorityBasedFileSystemView.java
new file mode 100644
index 0000000..e5528e1
--- /dev/null
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/PriorityBasedFileSystemView.java
@@ -0,0 +1,238 @@
+/*
+ * Copyright (c) 2019 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *          http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.uber.hoodie.common.table.view;
+
+import com.uber.hoodie.common.model.CompactionOperation;
+import com.uber.hoodie.common.model.FileSlice;
+import com.uber.hoodie.common.model.HoodieDataFile;
+import com.uber.hoodie.common.model.HoodieFileGroup;
+import com.uber.hoodie.common.table.HoodieTimeline;
+import com.uber.hoodie.common.table.SyncableFileSystemView;
+import com.uber.hoodie.common.table.timeline.HoodieInstant;
+import com.uber.hoodie.common.util.Functions.Function0;
+import com.uber.hoodie.common.util.Functions.Function1;
+import com.uber.hoodie.common.util.Functions.Function2;
+import com.uber.hoodie.common.util.Functions.Function3;
+import com.uber.hoodie.common.util.Option;
+import com.uber.hoodie.common.util.collection.Pair;
+import java.io.Serializable;
+import java.util.List;
+import java.util.stream.Stream;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+/**
+ * A file system view which proxies request to a preferred File System View implementation. In case of error,
+ * flip all subsequent calls to a backup file-system view implementation.
+ */
+public class PriorityBasedFileSystemView implements SyncableFileSystemView, Serializable {
+
+  private static Logger log = LogManager.getLogger(PriorityBasedFileSystemView.class);
+
+  private final SyncableFileSystemView preferredView;
+  private final SyncableFileSystemView secondaryView;
+  private boolean errorOnPreferredView;
+
+  public PriorityBasedFileSystemView(SyncableFileSystemView preferredView, SyncableFileSystemView secondaryView) {
+    this.preferredView = preferredView;
+    this.secondaryView = secondaryView;
+    this.errorOnPreferredView = false;
+  }
+
+  private <R> R execute(Function0<R> preferredFunction, Function0<R> secondaryFunction) {
+    if (errorOnPreferredView) {
+      log.warn("Routing request to secondary file-system view");
+      return secondaryFunction.apply();
+    } else {
+      try {
+        return preferredFunction.apply();
+      } catch (RuntimeException re) {
+        log.error("Got error running preferred function. Trying secondary", re);
+        errorOnPreferredView = true;
+        return secondaryFunction.apply();
+      }
+    }
+  }
+
+  private <T1, R> R execute(T1 val, Function1<T1, R> preferredFunction, Function1<T1, R> secondaryFunction) {
+    if (errorOnPreferredView) {
+      log.warn("Routing request to secondary file-system view");
+      return secondaryFunction.apply(val);
+    } else {
+      try {
+        return preferredFunction.apply(val);
+      } catch (RuntimeException re) {
+        log.error("Got error running preferred function. Trying secondary", re);
+        errorOnPreferredView = true;
+        return secondaryFunction.apply(val);
+      }
+    }
+  }
+
+  private <T1, T2, R> R execute(T1 val, T2 val2, Function2<T1, T2, R> preferredFunction,
+      Function2<T1, T2, R> secondaryFunction) {
+    if (errorOnPreferredView) {
+      log.warn("Routing request to secondary file-system view");
+      return secondaryFunction.apply(val, val2);
+    } else {
+      try {
+        return preferredFunction.apply(val, val2);
+      } catch (RuntimeException re) {
+        log.error("Got error running preferred function. Trying secondary", re);
+        errorOnPreferredView = true;
+        return secondaryFunction.apply(val, val2);
+      }
+    }
+  }
+
+  private <T1, T2, T3, R> R execute(T1 val, T2 val2, T3 val3, Function3<T1, T2, T3, R> preferredFunction,
+      Function3<T1, T2, T3, R> secondaryFunction) {
+    if (errorOnPreferredView) {
+      log.warn("Routing request to secondary file-system view");
+      return secondaryFunction.apply(val, val2, val3);
+    } else {
+      try {
+        return preferredFunction.apply(val, val2, val3);
+      } catch (RuntimeException re) {
+        log.error("Got error running preferred function. Trying secondary", re);
+        errorOnPreferredView = true;
+        return secondaryFunction.apply(val, val2, val3);
+      }
+    }
+  }
+
+  @Override
+  public Stream<HoodieDataFile> getLatestDataFiles(String partitionPath) {
+    return execute(partitionPath, preferredView::getLatestDataFiles, secondaryView::getLatestDataFiles);
+  }
+
+  @Override
+  public Stream<HoodieDataFile> getLatestDataFiles() {
+    return execute(preferredView::getLatestDataFiles, secondaryView::getLatestDataFiles);
+  }
+
+  @Override
+  public Stream<HoodieDataFile> getLatestDataFilesBeforeOrOn(String partitionPath, String maxCommitTime) {
+    return execute(partitionPath, maxCommitTime, preferredView::getLatestDataFilesBeforeOrOn,
+        secondaryView::getLatestDataFilesBeforeOrOn);
+  }
+
+  @Override
+  public Option<HoodieDataFile> getLatestDataFile(String partitionPath, String fileId) {
+    return execute(partitionPath, fileId, preferredView::getLatestDataFile, secondaryView::getLatestDataFile);
+  }
+
+  @Override
+  public Option<HoodieDataFile> getDataFileOn(String partitionPath, String instantTime, String fileId) {
+    return execute(partitionPath, instantTime, fileId, preferredView::getDataFileOn,
+        secondaryView::getDataFileOn);
+  }
+
+  @Override
+  public Stream<HoodieDataFile> getLatestDataFilesInRange(List<String> commitsToReturn) {
+    return execute(commitsToReturn, preferredView::getLatestDataFilesInRange, secondaryView::getLatestDataFilesInRange);
+  }
+
+  @Override
+  public Stream<HoodieDataFile> getAllDataFiles(String partitionPath) {
+    return execute(partitionPath, preferredView::getAllDataFiles, secondaryView::getAllDataFiles);
+  }
+
+  @Override
+  public Stream<FileSlice> getLatestFileSlices(String partitionPath) {
+    return execute(partitionPath, preferredView::getLatestFileSlices, secondaryView::getLatestFileSlices);
+  }
+
+  @Override
+  public Stream<FileSlice> getLatestUnCompactedFileSlices(String partitionPath) {
+    return execute(partitionPath, preferredView::getLatestUnCompactedFileSlices,
+        secondaryView::getLatestUnCompactedFileSlices);
+  }
+
+  @Override
+  public Stream<FileSlice> getLatestFileSlicesBeforeOrOn(String partitionPath, String maxCommitTime) {
+    return execute(partitionPath, maxCommitTime, preferredView::getLatestFileSlicesBeforeOrOn,
+        secondaryView::getLatestFileSlicesBeforeOrOn);
+  }
+
+  @Override
+  public Stream<FileSlice> getLatestMergedFileSlicesBeforeOrOn(String partitionPath, String maxInstantTime) {
+    return execute(partitionPath, maxInstantTime, preferredView::getLatestMergedFileSlicesBeforeOrOn,
+        secondaryView::getLatestMergedFileSlicesBeforeOrOn);
+  }
+
+  @Override
+  public Stream<FileSlice> getLatestFileSliceInRange(List<String> commitsToReturn) {
+    return execute(commitsToReturn, preferredView::getLatestFileSliceInRange, secondaryView::getLatestFileSliceInRange);
+  }
+
+  @Override
+  public Stream<FileSlice> getAllFileSlices(String partitionPath) {
+    return execute(partitionPath, preferredView::getAllFileSlices, secondaryView::getAllFileSlices);
+  }
+
+  @Override
+  public Stream<HoodieFileGroup> getAllFileGroups(String partitionPath) {
+    return execute(partitionPath, preferredView::getAllFileGroups, secondaryView::getAllFileGroups);
+  }
+
+  @Override
+  public Stream<Pair<String, CompactionOperation>> getPendingCompactionOperations() {
+    return execute(preferredView::getPendingCompactionOperations, secondaryView::getPendingCompactionOperations);
+  }
+
+  @Override
+  public void close() {
+    preferredView.close();
+    secondaryView.close();
+  }
+
+  @Override
+  public void reset() {
+    preferredView.reset();
+    secondaryView.reset();
+  }
+
+  @Override
+  public Option<HoodieInstant> getLastInstant() {
+    return execute(preferredView::getLastInstant, secondaryView::getLastInstant);
+  }
+
+  @Override
+  public HoodieTimeline getTimeline() {
+    return execute(preferredView::getTimeline, secondaryView::getTimeline);
+  }
+
+  @Override
+  public void sync() {
+    preferredView.reset();
+    secondaryView.reset();
+  }
+
+  @Override
+  public Option<FileSlice> getLatestFileSlice(String partitionPath, String fileId) {
+    return execute(partitionPath, fileId, preferredView::getLatestFileSlice, secondaryView::getLatestFileSlice);
+  }
+
+  public SyncableFileSystemView getPreferredView() {
+    return preferredView;
+  }
+
+  public SyncableFileSystemView getSecondaryView() {
+    return secondaryView;
+  }
+}
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/RemoteHoodieTableFileSystemView.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/RemoteHoodieTableFileSystemView.java
new file mode 100644
index 0000000..4dbb6e6
--- /dev/null
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/RemoteHoodieTableFileSystemView.java
@@ -0,0 +1,469 @@
+/*
+ * Copyright (c) 2019 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *          http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.uber.hoodie.common.table.view;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Preconditions;
+import com.uber.hoodie.common.model.CompactionOperation;
+import com.uber.hoodie.common.model.FileSlice;
+import com.uber.hoodie.common.model.HoodieDataFile;
+import com.uber.hoodie.common.model.HoodieFileGroup;
+import com.uber.hoodie.common.table.HoodieTableMetaClient;
+import com.uber.hoodie.common.table.HoodieTimeline;
+import com.uber.hoodie.common.table.SyncableFileSystemView;
+import com.uber.hoodie.common.table.timeline.HoodieInstant;
+import com.uber.hoodie.common.table.timeline.dto.CompactionOpDTO;
+import com.uber.hoodie.common.table.timeline.dto.DataFileDTO;
+import com.uber.hoodie.common.table.timeline.dto.FileGroupDTO;
+import com.uber.hoodie.common.table.timeline.dto.FileSliceDTO;
+import com.uber.hoodie.common.table.timeline.dto.InstantDTO;
+import com.uber.hoodie.common.table.timeline.dto.TimelineDTO;
+import com.uber.hoodie.common.util.Option;
+import com.uber.hoodie.common.util.StringUtils;
+import com.uber.hoodie.common.util.collection.Pair;
+import com.uber.hoodie.exception.HoodieRemoteException;
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Stream;
+import org.apache.http.client.fluent.Request;
+import org.apache.http.client.fluent.Response;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+/**
+ * A proxy for table file-system view which translates local View API calls to REST calls to remote timeline service
+ */
+public class RemoteHoodieTableFileSystemView implements SyncableFileSystemView, Serializable {
+
+  private static final String BASE_URL = "/v1/hoodie/view";
+  public static final String LATEST_PARTITION_SLICES_URL = String.format("%s/%s", BASE_URL,
+      "slices/partition/latest/");
+  public static final String LATEST_PARTITION_SLICE_URL = String.format("%s/%s", BASE_URL,
+      "slices/file/latest/");
+  public static final String LATEST_PARTITION_UNCOMPACTED_SLICES_URL = String.format("%s/%s", BASE_URL,
+      "slices/uncompacted/partition/latest/");
+  public static final String ALL_SLICES_URL = String.format("%s/%s", BASE_URL, "slices/all");
+  public static final String LATEST_SLICES_MERGED_BEFORE_ON_INSTANT_URL =
+      String.format("%s/%s", BASE_URL, "slices/merged/beforeoron/latest/");
+  public static final String LATEST_SLICES_RANGE_INSTANT_URL =
+      String.format("%s/%s", BASE_URL, "slices/range/latest/");
+  public static final String LATEST_SLICES_BEFORE_ON_INSTANT_URL =
+      String.format("%s/%s", BASE_URL, "slices/beforeoron/latest/");
+
+  public static final String PENDING_COMPACTION_OPS =
+      String.format("%s/%s", BASE_URL, "compactions/pending/");
+
+  public static final String LATEST_PARTITION_DATA_FILES_URL = String.format("%s/%s", BASE_URL,
+      "datafiles/latest/partition");
+  public static final String LATEST_PARTITION_DATA_FILE_URL = String.format("%s/%s", BASE_URL,
+      "datafile/latest/partition");
+  public static final String ALL_DATA_FILES = String.format("%s/%s", BASE_URL, "datafiles/all");
+  public static final String LATEST_ALL_DATA_FILES = String.format("%s/%s", BASE_URL, "datafiles/all/latest/");
+  public static final String LATEST_DATA_FILE_ON_INSTANT_URL =
+      String.format("%s/%s", BASE_URL, "datafile/on/latest/");
+
+  public static final String LATEST_DATA_FILES_RANGE_INSTANT_URL =
+      String.format("%s/%s", BASE_URL, "datafiles/range/latest/");
+  public static final String LATEST_DATA_FILES_BEFORE_ON_INSTANT_URL =
+      String.format("%s/%s", BASE_URL, "datafiles/beforeoron/latest/");
+
+  public static final String ALL_FILEGROUPS_FOR_PARTITION_URL =
+      String.format("%s/%s", BASE_URL, "filegroups/all/partition/");
+
+  public static final String LAST_INSTANT = String.format("%s/%s", BASE_URL, "timeline/instant/last");
+  public static final String LAST_INSTANTS = String.format("%s/%s", BASE_URL, "timeline/instants/last");
+
+  public static final String TIMELINE = String.format("%s/%s", BASE_URL, "timeline/instants/all");
+
+  // POST Requests
+  public static final String REFRESH_DATASET = String.format("%s/%s", BASE_URL, "refresh/");
+
+  public static final String PARTITION_PARAM = "partition";
+  public static final String BASEPATH_PARAM = "basepath";
+  public static final String INSTANT_PARAM = "instant";
+  public static final String MAX_INSTANT_PARAM = "maxinstant";
+  public static final String INSTANTS_PARAM = "instants";
+  public static final String FILEID_PARAM = "fileid";
+  public static final String LAST_INSTANT_TS = "lastinstantts";
+  public static final String TIMELINE_HASH = "timelinehash";
+  public static final String REFRESH_OFF = "refreshoff";
+
+  private static Logger log = LogManager.getLogger(RemoteHoodieTableFileSystemView.class);
+
+  private final String serverHost;
+  private final int serverPort;
+  private final String basePath;
+  private final HoodieTableMetaClient metaClient;
+  private final HoodieTimeline timeline;
+  private final ObjectMapper mapper;
+
+  private boolean closed = false;
+
+  private enum RequestMethod {
+    GET,
+    POST
+  }
+
+  public RemoteHoodieTableFileSystemView(String server, int port, HoodieTableMetaClient metaClient) {
+    this.basePath = metaClient.getBasePath();
+    this.serverHost = server;
+    this.serverPort = port;
+    this.mapper = new ObjectMapper();
+    this.metaClient = metaClient;
+    this.timeline = metaClient.getActiveTimeline().filterCompletedAndCompactionInstants();
+  }
+
+  private <T> T executeRequest(String requestPath, Map<String, String> queryParameters, TypeReference reference,
+      RequestMethod method) throws IOException {
+    Preconditions.checkArgument(!closed, "View already closed");
+
+    URIBuilder builder = new URIBuilder().setHost(serverHost).setPort(serverPort).setPath(requestPath)
+        .setScheme("http");
+
+    queryParameters.entrySet().stream().forEach(entry -> {
+      builder.addParameter(entry.getKey(), entry.getValue());
+    });
+
+    // Adding mandatory parameters - Last instants affecting file-slice
+    timeline.lastInstant().ifPresent(instant -> builder.addParameter(LAST_INSTANT_TS, instant.getTimestamp()));
+    builder.addParameter(TIMELINE_HASH, timeline.getTimelineHash());
+
+    String url = builder.toString();
+    log.info("Sending request : (" + url + ")");
+    Response response = null;
+    int timeout = 1000 * 300; // 5 min timeout
+    switch (method) {
+      case GET:
+        response = Request.Get(url).connectTimeout(timeout).socketTimeout(timeout).execute();
+        break;
+      case POST:
+      default:
+        response = Request.Post(url).connectTimeout(timeout).socketTimeout(timeout).execute();
+        break;
+    }
+    String content = response.returnContent().asString();
+    return mapper.readValue(content, reference);
+  }
+
+  private Map<String, String> getParamsWithPartitionPath(String partitionPath) {
+    Map<String, String> paramsMap = new HashMap<>();
+    paramsMap.put(BASEPATH_PARAM, basePath);
+    paramsMap.put(PARTITION_PARAM, partitionPath);
+    return paramsMap;
+  }
+
+  private Map<String, String> getParams() {
+    Map<String, String> paramsMap = new HashMap<>();
+    paramsMap.put(BASEPATH_PARAM, basePath);
+    return paramsMap;
+  }
+
+  private Map<String, String> getParams(String paramName, String instant) {
+    Map<String, String> paramsMap = new HashMap<>();
+    paramsMap.put(BASEPATH_PARAM, basePath);
+    paramsMap.put(paramName, instant);
+    return paramsMap;
+  }
+
+  private Map<String, String> getParamsWithAdditionalParam(String partitionPath, String paramName, String paramVal) {
+    Map<String, String> paramsMap = new HashMap<>();
+    paramsMap.put(BASEPATH_PARAM, basePath);
+    paramsMap.put(PARTITION_PARAM, partitionPath);
+    paramsMap.put(paramName, paramVal);
+    return paramsMap;
+  }
+
+  private Map<String, String> getParamsWithAdditionalParams(String partitionPath, String[] paramNames,
+      String[] paramVals) {
+    Map<String, String> paramsMap = new HashMap<>();
+    paramsMap.put(BASEPATH_PARAM, basePath);
+    paramsMap.put(PARTITION_PARAM, partitionPath);
+    Preconditions.checkArgument(paramNames.length == paramVals.length);
+    for (int i = 0; i < paramNames.length; i++) {
+      paramsMap.put(paramNames[i], paramVals[i]);
+    }
+    return paramsMap;
+  }
+
+  @Override
+  public Stream<HoodieDataFile> getLatestDataFiles(String partitionPath) {
+    Map<String, String> paramsMap = getParamsWithPartitionPath(partitionPath);
+    try {
+      List<DataFileDTO> dataFiles = executeRequest(LATEST_PARTITION_DATA_FILES_URL, paramsMap,
+          new TypeReference<List<DataFileDTO>>() {
+          }, RequestMethod.GET);
+      return dataFiles.stream().map(DataFileDTO::toHoodieDataFile);
+    } catch (IOException e) {
+      throw new HoodieRemoteException(e);
+    }
+  }
+
+  @Override
+  public Stream<HoodieDataFile> getLatestDataFiles() {
+    Map<String, String> paramsMap = getParams();
+    try {
+      List<DataFileDTO> dataFiles = executeRequest(LATEST_ALL_DATA_FILES, paramsMap,
+          new TypeReference<List<DataFileDTO>>() {
+          }, RequestMethod.GET);
+      return dataFiles.stream().map(DataFileDTO::toHoodieDataFile);
+    } catch (IOException e) {
+      throw new HoodieRemoteException(e);
+    }
+  }
+
+  @Override
+  public Stream<HoodieDataFile> getLatestDataFilesBeforeOrOn(String partitionPath, String maxCommitTime) {
+    Map<String, String> paramsMap = getParamsWithAdditionalParam(partitionPath, MAX_INSTANT_PARAM, maxCommitTime);
+    try {
+      List<DataFileDTO> dataFiles = executeRequest(LATEST_DATA_FILES_BEFORE_ON_INSTANT_URL, paramsMap,
+          new TypeReference<List<DataFileDTO>>() {
+          }, RequestMethod.GET);
+      return dataFiles.stream().map(DataFileDTO::toHoodieDataFile);
+    } catch (IOException e) {
+      throw new HoodieRemoteException(e);
+    }
+  }
+
+  @Override
+  public Option<HoodieDataFile> getDataFileOn(String partitionPath, String instantTime, String fileId) {
+    Map<String, String> paramsMap = getParamsWithAdditionalParams(partitionPath,
+        new String[]{INSTANT_PARAM, FILEID_PARAM},
+        new String[]{instantTime, fileId});
+    try {
+      List<DataFileDTO> dataFiles = executeRequest(LATEST_DATA_FILE_ON_INSTANT_URL, paramsMap,
+          new TypeReference<List<DataFileDTO>>() {
+          }, RequestMethod.GET);
+      return Option.fromJavaOptional(dataFiles.stream().map(DataFileDTO::toHoodieDataFile).findFirst());
+    } catch (IOException e) {
+      throw new HoodieRemoteException(e);
+    }
+  }
+
+  @Override
+  public Stream<HoodieDataFile> getLatestDataFilesInRange(List<String> commitsToReturn) {
+    Map<String, String> paramsMap = getParams(INSTANTS_PARAM,
+        StringUtils.join(commitsToReturn.toArray(new String[0]), ","));
+    try {
+      List<DataFileDTO> dataFiles = executeRequest(LATEST_DATA_FILES_RANGE_INSTANT_URL, paramsMap,
+          new TypeReference<List<DataFileDTO>>() {
+          }, RequestMethod.GET);
+      return dataFiles.stream().map(DataFileDTO::toHoodieDataFile);
+    } catch (IOException e) {
+      throw new HoodieRemoteException(e);
+    }
+  }
+
+  @Override
+  public Stream<HoodieDataFile> getAllDataFiles(String partitionPath) {
+    Map<String, String> paramsMap = getParamsWithPartitionPath(partitionPath);
+    try {
+      List<DataFileDTO> dataFiles = executeRequest(ALL_DATA_FILES, paramsMap,
+          new TypeReference<List<DataFileDTO>>() {
+          }, RequestMethod.GET);
+      return dataFiles.stream().map(DataFileDTO::toHoodieDataFile);
+    } catch (IOException e) {
+      throw new HoodieRemoteException(e);
+    }
+  }
+
+  @Override
+  public Stream<FileSlice> getLatestFileSlices(String partitionPath) {
+    Map<String, String> paramsMap = getParamsWithPartitionPath(partitionPath);
+    try {
+      List<FileSliceDTO> dataFiles = executeRequest(LATEST_PARTITION_SLICES_URL, paramsMap,
+          new TypeReference<List<FileSliceDTO>>() {
+          }, RequestMethod.GET);
+      return dataFiles.stream().map(FileSliceDTO::toFileSlice);
+    } catch (IOException e) {
+      throw new HoodieRemoteException(e);
+    }
+  }
+
+  @Override
+  public Option<FileSlice> getLatestFileSlice(String partitionPath, String fileId) {
+    Map<String, String> paramsMap = getParamsWithAdditionalParam(partitionPath, FILEID_PARAM, fileId);
+    try {
+      List<FileSliceDTO> dataFiles = executeRequest(LATEST_PARTITION_SLICE_URL, paramsMap,
+          new TypeReference<List<FileSliceDTO>>() {
+          }, RequestMethod.GET);
+      return Option.fromJavaOptional(dataFiles.stream().map(FileSliceDTO::toFileSlice).findFirst());
+    } catch (IOException e) {
+      throw new HoodieRemoteException(e);
+    }
+  }
+
+  @Override
+  public Stream<FileSlice> getLatestUnCompactedFileSlices(String partitionPath) {
+    Map<String, String> paramsMap = getParamsWithPartitionPath(partitionPath);
+    try {
+      List<FileSliceDTO> dataFiles = executeRequest(LATEST_PARTITION_UNCOMPACTED_SLICES_URL, paramsMap,
+          new TypeReference<List<FileSliceDTO>>() {
+          }, RequestMethod.GET);
+      return dataFiles.stream().map(FileSliceDTO::toFileSlice);
+    } catch (IOException e) {
+      throw new HoodieRemoteException(e);
+    }
+  }
+
+  @Override
+  public Stream<FileSlice> getLatestFileSlicesBeforeOrOn(String partitionPath, String maxCommitTime) {
+    Map<String, String> paramsMap = getParamsWithAdditionalParam(partitionPath, MAX_INSTANT_PARAM, maxCommitTime);
+    try {
+      List<FileSliceDTO> dataFiles = executeRequest(LATEST_SLICES_BEFORE_ON_INSTANT_URL, paramsMap,
+          new TypeReference<List<FileSliceDTO>>() {
+          }, RequestMethod.GET);
+      return dataFiles.stream().map(FileSliceDTO::toFileSlice);
+    } catch (IOException e) {
+      throw new HoodieRemoteException(e);
+    }
+  }
+
+  @Override
+  public Stream<FileSlice> getLatestMergedFileSlicesBeforeOrOn(String partitionPath, String maxInstantTime) {
+    Map<String, String> paramsMap = getParamsWithAdditionalParam(partitionPath, MAX_INSTANT_PARAM, maxInstantTime);
+    try {
+      List<FileSliceDTO> dataFiles = executeRequest(LATEST_SLICES_MERGED_BEFORE_ON_INSTANT_URL, paramsMap,
+          new TypeReference<List<FileSliceDTO>>() {
+          }, RequestMethod.GET);
+      return dataFiles.stream().map(FileSliceDTO::toFileSlice);
+    } catch (IOException e) {
+      throw new HoodieRemoteException(e);
+    }
+  }
+
+  @Override
+  public Stream<FileSlice> getLatestFileSliceInRange(List<String> commitsToReturn) {
+    Map<String, String> paramsMap = getParams(INSTANTS_PARAM,
+        StringUtils.join(commitsToReturn.toArray(new String[0]), ","));
+    try {
+      List<FileSliceDTO> dataFiles = executeRequest(LATEST_SLICES_RANGE_INSTANT_URL, paramsMap,
+          new TypeReference<List<FileSliceDTO>>() {
+          }, RequestMethod.GET);
+      return dataFiles.stream().map(FileSliceDTO::toFileSlice);
+    } catch (IOException e) {
+      throw new HoodieRemoteException(e);
+    }
+  }
+
+  @Override
+  public Stream<FileSlice> getAllFileSlices(String partitionPath) {
+    Map<String, String> paramsMap = getParamsWithPartitionPath(partitionPath);
+    try {
+      List<FileSliceDTO> dataFiles = executeRequest(ALL_SLICES_URL, paramsMap,
+          new TypeReference<List<FileSliceDTO>>() {
+          }, RequestMethod.GET);
+      return dataFiles.stream().map(FileSliceDTO::toFileSlice);
+    } catch (IOException e) {
+      throw new HoodieRemoteException(e);
+    }
+  }
+
+  @Override
+  public Stream<HoodieFileGroup> getAllFileGroups(String partitionPath) {
+    Map<String, String> paramsMap = getParamsWithPartitionPath(partitionPath);
+    try {
+      List<FileGroupDTO> fileGroups = executeRequest(ALL_FILEGROUPS_FOR_PARTITION_URL, paramsMap,
+          new TypeReference<List<FileGroupDTO>>() {
+          }, RequestMethod.GET);
+      return fileGroups.stream().map(dto -> FileGroupDTO.toFileGroup(dto, metaClient));
+    } catch (IOException e) {
+      throw new HoodieRemoteException(e);
+    }
+  }
+
+  public boolean refresh() {
+    Map<String, String> paramsMap = getParams();
+    try {
+      return executeRequest(REFRESH_DATASET, paramsMap, new TypeReference<Boolean>() {
+      }, RequestMethod.POST);
+    } catch (IOException e) {
+      throw new HoodieRemoteException(e);
+    }
+  }
+
+  @Override
+  public Stream<Pair<String, CompactionOperation>> getPendingCompactionOperations() {
+    Map<String, String> paramsMap = getParams();
+    try {
+      List<CompactionOpDTO> dtos = executeRequest(PENDING_COMPACTION_OPS, paramsMap,
+          new TypeReference<List<CompactionOpDTO>>() {
+          }, RequestMethod.GET);
+      return dtos.stream().map(CompactionOpDTO::toCompactionOperation);
+    } catch (IOException e) {
+      throw new HoodieRemoteException(e);
+    }
+  }
+
+  @Override
+  public void close() {
+    closed = true;
+  }
+
+  @Override
+  public void reset() {
+    refresh();
+  }
+
+  @Override
+  public Option<HoodieInstant> getLastInstant() {
+    Map<String, String> paramsMap = getParams();
+    try {
+      List<InstantDTO> instants = executeRequest(LAST_INSTANT, paramsMap,
+          new TypeReference<List<InstantDTO>>() {
+          }, RequestMethod.GET);
+      return Option.fromJavaOptional(instants.stream().map(InstantDTO::toInstant).findFirst());
+    } catch (IOException e) {
+      throw new HoodieRemoteException(e);
+    }
+  }
+
+  @Override
+  public HoodieTimeline getTimeline() {
+    Map<String, String> paramsMap = getParams();
+    try {
+      TimelineDTO timeline = executeRequest(TIMELINE, paramsMap,
+          new TypeReference<TimelineDTO>() {
+          }, RequestMethod.GET);
+      return TimelineDTO.toTimeline(timeline, metaClient);
+    } catch (IOException e) {
+      throw new HoodieRemoteException(e);
+    }
+  }
+
+  @Override
+  public void sync() {
+    //noop
+  }
+
+  @Override
+  public Option<HoodieDataFile> getLatestDataFile(String partitionPath, String fileId) {
+    Map<String, String> paramsMap = getParamsWithAdditionalParam(partitionPath, FILEID_PARAM, fileId);
+    try {
+      List<DataFileDTO> dataFiles = executeRequest(LATEST_PARTITION_DATA_FILE_URL, paramsMap,
+          new TypeReference<List<DataFileDTO>>() {
+          }, RequestMethod.GET);
+      return Option.fromJavaOptional(dataFiles.stream().map(DataFileDTO::toHoodieDataFile).findFirst());
+    } catch (IOException e) {
+      throw new HoodieRemoteException(e);
+    }
+  }
+}
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/RocksDbBasedFileSystemView.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/RocksDbBasedFileSystemView.java
new file mode 100644
index 0000000..a48d10f
--- /dev/null
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/RocksDbBasedFileSystemView.java
@@ -0,0 +1,341 @@
+/*
+ *  Copyright (c) 2019 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  You may obtain a copy of the License at
+ *
+ *           http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package com.uber.hoodie.common.table.view;
+
+import com.google.common.base.Preconditions;
+import com.uber.hoodie.common.model.CompactionOperation;
+import com.uber.hoodie.common.model.FileSlice;
+import com.uber.hoodie.common.model.HoodieDataFile;
+import com.uber.hoodie.common.model.HoodieFileGroup;
+import com.uber.hoodie.common.model.HoodieFileGroupId;
+import com.uber.hoodie.common.model.HoodieLogFile;
+import com.uber.hoodie.common.table.HoodieTableMetaClient;
+import com.uber.hoodie.common.table.HoodieTimeline;
+import com.uber.hoodie.common.util.Option;
+import com.uber.hoodie.common.util.RocksDBDAO;
+import com.uber.hoodie.common.util.RocksDBSchemaHelper;
+import com.uber.hoodie.common.util.collection.Pair;
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+/**
+ * A file-system view implementation on top of embedded Rocks DB store.
+ * For each DataSet : 3 column Family is added for storing
+ *  (1) File-Slices and Data Files for View lookups
+ *  (2) Pending compaction operations
+ *  (3) Partitions tracked
+ *
+ *  Fine-grained retrieval API to fetch latest file-slice and data-file which are common operations
+ *  for ingestion/compaction are supported.
+ *
+ *  TODO: vb The current implementation works in embedded server mode where each restarts blows away the view stores.
+ *  To support view-state preservation across restarts, Hoodie timeline also needs to be stored
+ *  inorder to detect changes to timeline across restarts.
+ */
+public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSystemView {
+
+  private static Logger log = LogManager.getLogger(RocksDbBasedFileSystemView.class);
+
+  private final FileSystemViewStorageConfig config;
+
+  private final RocksDBSchemaHelper schemaHelper;
+
+  private RocksDBDAO rocksDB;
+
+  private boolean closed = false;
+
+  public RocksDbBasedFileSystemView(HoodieTableMetaClient metaClient,
+      HoodieTimeline visibleActiveTimeline, FileSystemViewStorageConfig config) {
+    super(config.isIncrementalTimelineSyncEnabled());
+    this.config = config;
+    this.schemaHelper = new RocksDBSchemaHelper(metaClient);
+    this.rocksDB = new RocksDBDAO(metaClient.getBasePath(), config);
+    init(metaClient, visibleActiveTimeline);
+  }
+
+  public RocksDbBasedFileSystemView(HoodieTableMetaClient metaClient,
+      HoodieTimeline visibleActiveTimeline, FileStatus[] fileStatuses, FileSystemViewStorageConfig config) {
+    this(metaClient, visibleActiveTimeline, config);
+    addFilesToView(fileStatuses);
+  }
+
+  @Override
+  protected void init(HoodieTableMetaClient metaClient, HoodieTimeline visibleActiveTimeline) {
+    schemaHelper.getAllColumnFamilies().stream().forEach(rocksDB::addColumnFamily);
+    super.init(metaClient, visibleActiveTimeline);
+    log.info("Created ROCKSDB based file-system view at " + config.getRocksdbBasePath());
+  }
+
+  @Override
+  protected boolean isPendingCompactionScheduledForFileId(HoodieFileGroupId fgId) {
+    return getPendingCompactionOperationWithInstant(fgId).isPresent();
+  }
+
+  @Override
+  protected void resetPendingCompactionOperations(Stream<Pair<String, CompactionOperation>> operations) {
+    rocksDB.writeBatch(batch -> {
+      operations.forEach(opPair -> {
+        rocksDB.putInBatch(batch, schemaHelper.getColFamilyForPendingCompaction(),
+            schemaHelper.getKeyForPendingCompactionLookup(opPair.getValue().getFileGroupId()), opPair);
+      });
+      log.info("Initializing pending compaction operations. Count=" + batch.count());
+    });
+  }
+
+  @Override
+  protected void addPendingCompactionOperations(Stream<Pair<String, CompactionOperation>> operations) {
+    rocksDB.writeBatch(batch -> {
+      operations.forEach(opInstantPair -> {
+        Preconditions.checkArgument(!isPendingCompactionScheduledForFileId(opInstantPair.getValue().getFileGroupId()),
+            "Duplicate FileGroupId found in pending compaction operations. FgId :"
+                + opInstantPair.getValue().getFileGroupId());
+        rocksDB.putInBatch(batch, schemaHelper.getColFamilyForPendingCompaction(),
+            schemaHelper.getKeyForPendingCompactionLookup(opInstantPair.getValue().getFileGroupId()), opInstantPair);
+      });
+    });
+  }
+
+  @Override
+  void removePendingCompactionOperations(Stream<Pair<String, CompactionOperation>> operations) {
+    rocksDB.writeBatch(batch -> {
+      operations.forEach(opInstantPair -> {
+        Preconditions.checkArgument(
+            getPendingCompactionOperationWithInstant(opInstantPair.getValue().getFileGroupId()) != null,
+            "Trying to remove a FileGroupId which is not found in pending compaction operations. FgId :"
+                + opInstantPair.getValue().getFileGroupId());
+        rocksDB.deleteInBatch(batch, schemaHelper.getColFamilyForPendingCompaction(),
+            schemaHelper.getKeyForPendingCompactionLookup(opInstantPair.getValue().getFileGroupId()));
+      });
+    });
+  }
+
+  @Override
+  protected void resetViewState() {
+    log.info("Deleting all rocksdb data associated with dataset filesystem view");
+    rocksDB.close();
+    rocksDB = new RocksDBDAO(metaClient.getBasePath(), config);
+  }
+
+  @Override
+  protected Option<Pair<String, CompactionOperation>> getPendingCompactionOperationWithInstant(HoodieFileGroupId fgId) {
+    String lookupKey = schemaHelper.getKeyForPendingCompactionLookup(fgId);
+    Pair<String, CompactionOperation> instantOperationPair =
+        rocksDB.get(schemaHelper.getColFamilyForPendingCompaction(), lookupKey);
+    return Option.ofNullable(instantOperationPair);
+  }
+
+  @Override
+  protected boolean isPartitionAvailableInStore(String partitionPath) {
+    String lookupKey = schemaHelper.getKeyForPartitionLookup(partitionPath);
+    Serializable obj = rocksDB.get(schemaHelper.getColFamilyForStoredPartitions(), lookupKey);
+    return obj != null;
+  }
+
+  @Override
+  protected void storePartitionView(String partitionPath, List<HoodieFileGroup> fileGroups) {
+    log.info("Resetting and adding new partition (" + partitionPath + ") to ROCKSDB based file-system view at "
+        + config.getRocksdbBasePath() + ", Total file-groups=" + fileGroups.size());
+
+    String lookupKey = schemaHelper.getKeyForPartitionLookup(partitionPath);
+    rocksDB.delete(schemaHelper.getColFamilyForStoredPartitions(), lookupKey);
+
+    // First delete partition views
+    rocksDB.prefixDelete(schemaHelper.getColFamilyForView(),
+        schemaHelper.getPrefixForSliceViewByPartition(partitionPath));
+    rocksDB.prefixDelete(schemaHelper.getColFamilyForView(),
+        schemaHelper.getPrefixForDataFileViewByPartition(partitionPath));
+
+    // Now add them
+    fileGroups.stream().forEach(fg -> {
+      rocksDB.writeBatch(batch -> {
+        fg.getAllFileSlicesIncludingInflight().forEach(fs -> {
+          rocksDB.putInBatch(batch, schemaHelper.getColFamilyForView(), schemaHelper.getKeyForSliceView(fg, fs), fs);
+          fs.getDataFile().ifPresent(df -> {
+            rocksDB.putInBatch(batch, schemaHelper.getColFamilyForView(), schemaHelper.getKeyForDataFileView(fg, fs),
+                df);
+          });
+        });
+      });
+    });
+
+    // record that partition is loaded.
+    rocksDB.put(schemaHelper.getColFamilyForStoredPartitions(), lookupKey, Boolean.TRUE);
+    log.info("Finished adding new partition (" + partitionPath + ") to ROCKSDB based file-system view at "
+        + config.getRocksdbBasePath() + ", Total file-groups=" + fileGroups.size());
+  }
+
+  @Override
+  /**
+   * This is overridden to incrementally apply file-slices to rocks DB
+   */
+  protected void applyDeltaFileSlicesToPartitionView(String partition, List<HoodieFileGroup> deltaFileGroups,
+      DeltaApplyMode mode) {
+    rocksDB.writeBatch(batch -> {
+      deltaFileGroups.stream().forEach(fg -> {
+        fg.getAllRawFileSlices().map(fs -> {
+          FileSlice oldSlice = getFileSlice(partition, fs.getFileId(), fs.getBaseInstantTime());
+          if (null == oldSlice) {
+            return fs;
+          } else {
+            // First remove the file-slice
+            log.info("Removing old Slice in DB. FS=" + oldSlice);
+            rocksDB.deleteInBatch(batch, schemaHelper.getColFamilyForView(),
+                schemaHelper.getKeyForSliceView(fg, oldSlice));
+            rocksDB.deleteInBatch(batch, schemaHelper.getColFamilyForView(),
+                schemaHelper.getKeyForDataFileView(fg, oldSlice));
+
+            Map<String, HoodieLogFile> logFiles = oldSlice.getLogFiles()
+                .map(lf -> Pair.of(Path.getPathWithoutSchemeAndAuthority(lf.getPath()).toString(), lf))
+                .collect(Collectors.toMap(Pair::getKey, Pair::getValue));
+            Map<String, HoodieLogFile> deltaLogFiles = fs.getLogFiles()
+                .map(lf -> Pair.of(Path.getPathWithoutSchemeAndAuthority(lf.getPath()).toString(), lf))
+                .collect(Collectors.toMap(Pair::getKey, Pair::getValue));
+
+            switch (mode) {
+              case ADD: {
+                FileSlice newFileSlice = new FileSlice(oldSlice.getFileGroupId(), oldSlice.getBaseInstantTime());
+                oldSlice.getDataFile().ifPresent(df -> newFileSlice.setDataFile(df));
+                fs.getDataFile().ifPresent(df -> newFileSlice.setDataFile(df));
+                Map<String, HoodieLogFile> newLogFiles = new HashMap<>(logFiles);
+                deltaLogFiles.entrySet().stream().filter(e -> !logFiles.containsKey(e.getKey()))
+                    .forEach(p -> newLogFiles.put(p.getKey(), p.getValue()));
+                newLogFiles.values().stream().forEach(lf -> newFileSlice.addLogFile(lf));
+                log.info("Adding back new File Slice after add FS=" + newFileSlice);
+                return newFileSlice;
+              }
+              case REMOVE: {
+                log.info("Removing old File Slice =" + fs);
+                FileSlice newFileSlice = new FileSlice(oldSlice.getFileGroupId(), oldSlice.getBaseInstantTime());
+                fs.getDataFile().orElseGet(() -> {
+                  oldSlice.getDataFile().ifPresent(df -> newFileSlice.setDataFile(df));
+                  return null;
+                });
+
+                deltaLogFiles.keySet().stream().forEach(p -> logFiles.remove(p));
+                //Add remaining log files back
+                logFiles.values().stream().forEach(lf -> newFileSlice.addLogFile(lf));
+                if (newFileSlice.getDataFile().isPresent() || (newFileSlice.getLogFiles().count() > 0)) {
+                  log.info("Adding back new file-slice after remove FS=" + newFileSlice);
+                  return newFileSlice;
+                }
+                return null;
+              }
+              default:
+                throw new IllegalStateException("Unknown diff apply mode=" + mode);
+            }
+          }
+        }).filter(Objects::nonNull).forEach(fs -> {
+          rocksDB.putInBatch(batch, schemaHelper.getColFamilyForView(), schemaHelper.getKeyForSliceView(fg, fs), fs);
+          fs.getDataFile().ifPresent(df -> {
+            rocksDB.putInBatch(batch, schemaHelper.getColFamilyForView(), schemaHelper.getKeyForDataFileView(fg, fs),
+                df);
+          });
+        });
+      });
+    });
+  }
+
+  @Override
+  Stream<Pair<String, CompactionOperation>> fetchPendingCompactionOperations() {
+    return rocksDB.<Pair<String, CompactionOperation>>prefixSearch(
+        schemaHelper.getColFamilyForPendingCompaction(), "").map(Pair::getValue);
+  }
+
+  @Override
+  Stream<HoodieDataFile> fetchAllDataFiles(String partitionPath) {
+    return rocksDB.<HoodieDataFile>prefixSearch(schemaHelper.getColFamilyForView(),
+        schemaHelper.getPrefixForDataFileViewByPartition(partitionPath))
+        .map(Pair::getValue);
+  }
+
+  @Override
+  Stream<HoodieFileGroup> fetchAllStoredFileGroups(String partitionPath) {
+    return getFileGroups(rocksDB.<FileSlice>prefixSearch(schemaHelper.getColFamilyForView(),
+        schemaHelper.getPrefixForSliceViewByPartition(partitionPath)).map(Pair::getValue));
+  }
+
+  @Override
+  Stream<HoodieFileGroup> fetchAllStoredFileGroups() {
+    return getFileGroups(rocksDB.<FileSlice>prefixSearch(schemaHelper.getColFamilyForView(),
+        schemaHelper.getPrefixForSliceView()).map(Pair::getValue));
+  }
+
+  @Override
+  protected Option<FileSlice> fetchLatestFileSlice(String partitionPath, String fileId) {
+    // Retries only file-slices of the file and filters for the latest
+    return Option.ofNullable(rocksDB.<FileSlice>prefixSearch(schemaHelper.getColFamilyForView(),
+        schemaHelper.getPrefixForSliceViewByPartitionFile(partitionPath, fileId))
+        .map(Pair::getValue)
+        .reduce(null, (x, y) ->
+            ((x == null) ? y : (y == null) ? null : HoodieTimeline.compareTimestamps(x.getBaseInstantTime(),
+                y.getBaseInstantTime(), HoodieTimeline.GREATER) ? x : y)));
+  }
+
+  @Override
+  protected Option<HoodieDataFile> fetchLatestDataFile(String partitionPath, String fileId) {
+    // Retries only file-slices of the file and filters for the latest
+    return Option.ofNullable(rocksDB.<HoodieDataFile>prefixSearch(schemaHelper.getColFamilyForView(),
+        schemaHelper.getPrefixForDataFileViewByPartitionFile(partitionPath, fileId))
+        .map(Pair::getValue)
+        .reduce(null, (x, y) ->
+            ((x == null) ? y : (y == null) ? null : HoodieTimeline.compareTimestamps(x.getCommitTime(),
+                y.getCommitTime(), HoodieTimeline.GREATER) ? x : y)));
+  }
+
+  @Override
+  Option<HoodieFileGroup> fetchHoodieFileGroup(String partitionPath, String fileId)  {
+    return Option.fromJavaOptional(
+        getFileGroups(rocksDB.<FileSlice>prefixSearch(schemaHelper.getColFamilyForView(),
+            schemaHelper.getPrefixForSliceViewByPartitionFile(partitionPath, fileId))
+            .map(Pair::getValue)).findFirst());
+  }
+
+  private Stream<HoodieFileGroup> getFileGroups(Stream<FileSlice> sliceStream) {
+    return sliceStream.map(s -> Pair.of(Pair.of(s.getPartitionPath(), s.getFileId()), s))
+        .collect(Collectors.groupingBy(Pair::getKey)).entrySet().stream().map(slicePair -> {
+          HoodieFileGroup fg =
+              new HoodieFileGroup(slicePair.getKey().getKey(), slicePair.getKey().getValue(), visibleActiveTimeline);
+          slicePair.getValue().forEach(e -> fg.addFileSlice(e.getValue()));
+          return fg;
+        });
+  }
+
+  private FileSlice getFileSlice(String partitionPath, String fileId, String instantTime) {
+    String key = schemaHelper.getKeyForSliceView(partitionPath, fileId, instantTime);
+    return rocksDB.<FileSlice>get(schemaHelper.getColFamilyForView(), key);
+  }
+
+  @Override
+  public void close() {
+    closed = true;
+    rocksDB.close();
+  }
+
+  @Override
+  boolean isClosed() {
+    return closed;
+  }
+}
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/SpillableMapBasedFileSystemView.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/SpillableMapBasedFileSystemView.java
new file mode 100644
index 0000000..3c6761c
--- /dev/null
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/SpillableMapBasedFileSystemView.java
@@ -0,0 +1,109 @@
+/*
+ * Copyright (c) 2019 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *          http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.uber.hoodie.common.table.view;
+
+import com.uber.hoodie.common.model.CompactionOperation;
+import com.uber.hoodie.common.model.HoodieFileGroup;
+import com.uber.hoodie.common.model.HoodieFileGroupId;
+import com.uber.hoodie.common.table.HoodieTableMetaClient;
+import com.uber.hoodie.common.table.HoodieTimeline;
+import com.uber.hoodie.common.util.DefaultSizeEstimator;
+import com.uber.hoodie.common.util.collection.ExternalSpillableMap;
+import com.uber.hoodie.common.util.collection.Pair;
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Stream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+/**
+ * Table FileSystemView implementation where view is stored in spillable disk using fixed memory
+ */
+public class SpillableMapBasedFileSystemView extends HoodieTableFileSystemView {
+
+  private static Logger log = LogManager.getLogger(SpillableMapBasedFileSystemView.class);
+
+  private final long maxMemoryForFileGroupMap;
+  private final long maxMemoryForPendingCompaction;
+  private final String baseStoreDir;
+
+  public SpillableMapBasedFileSystemView(HoodieTableMetaClient metaClient,
+      HoodieTimeline visibleActiveTimeline, FileSystemViewStorageConfig config) {
+    super(config.isIncrementalTimelineSyncEnabled());
+    this.maxMemoryForFileGroupMap = config.getMaxMemoryForFileGroupMap();
+    this.maxMemoryForPendingCompaction = config.getMaxMemoryForPendingCompaction();
+    this.baseStoreDir = config.getBaseStoreDir();
+    init(metaClient, visibleActiveTimeline);
+  }
+
+  public SpillableMapBasedFileSystemView(HoodieTableMetaClient metaClient,
+      HoodieTimeline visibleActiveTimeline, FileStatus[] fileStatuses, FileSystemViewStorageConfig config) {
+    this(metaClient, visibleActiveTimeline, config);
+    addFilesToView(fileStatuses);
+  }
+
+  @Override
+  protected Map<String, List<HoodieFileGroup>> createPartitionToFileGroups() {
+    try {
+      log.info("Creating Partition To File groups map using external spillable Map. Max Mem="
+          + maxMemoryForFileGroupMap + ", BaseDir=" + baseStoreDir);
+      new File(baseStoreDir).mkdirs();
+      return (Map<String, List<HoodieFileGroup>>)
+          (new ExternalSpillableMap<>(maxMemoryForFileGroupMap, baseStoreDir, new DefaultSizeEstimator(),
+              new DefaultSizeEstimator<>()));
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  protected Map<HoodieFileGroupId, Pair<String, CompactionOperation>> createFileIdToPendingCompactionMap(
+      Map<HoodieFileGroupId, Pair<String, CompactionOperation>> fgIdToPendingCompaction) {
+    try {
+      log.info("Creating Pending Compaction map using external spillable Map. Max Mem="
+          + maxMemoryForPendingCompaction + ", BaseDir=" + baseStoreDir);
+      new File(baseStoreDir).mkdirs();
+      Map<HoodieFileGroupId, Pair<String, CompactionOperation>> pendingMap =
+          new ExternalSpillableMap<>(maxMemoryForPendingCompaction, baseStoreDir, new DefaultSizeEstimator(),
+              new DefaultSizeEstimator<>());
+      pendingMap.putAll(fgIdToPendingCompaction);
+      return pendingMap;
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  public Stream<HoodieFileGroup> getAllFileGroups() {
+    return ((ExternalSpillableMap)partitionToFileGroupsMap).valueStream()
+        .flatMap(fg -> ((List<HoodieFileGroup>)fg).stream());
+  }
+
+  @Override
+  Stream<Pair<String, CompactionOperation>> fetchPendingCompactionOperations() {
+    return ((ExternalSpillableMap)fgIdToPendingCompaction).valueStream();
+
+  }
+
+  @Override
+  public Stream<HoodieFileGroup> fetchAllStoredFileGroups() {
+    return ((ExternalSpillableMap)partitionToFileGroupsMap).valueStream().flatMap(fg -> {
+      return ((List<HoodieFileGroup>)fg).stream();
+    });
+  }
+}
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/CompactionUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/CompactionUtils.java
index f3552b4..589db06 100644
--- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/CompactionUtils.java
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/CompactionUtils.java
@@ -147,17 +147,7 @@ public class CompactionUtils {
     Map<HoodieFileGroupId, Pair<String, HoodieCompactionOperation>> fgIdToPendingCompactionWithInstantMap =
         new HashMap<>();
     pendingCompactionPlanWithInstants.stream().flatMap(instantPlanPair -> {
-      HoodieInstant instant = instantPlanPair.getKey();
-      HoodieCompactionPlan compactionPlan = instantPlanPair.getValue();
-      List<HoodieCompactionOperation> ops = compactionPlan.getOperations();
-      if (null != ops) {
-        return ops.stream().map(op -> {
-          return Pair.of(new HoodieFileGroupId(op.getPartitionPath(), op.getFileId()),
-              Pair.of(instant.getTimestamp(), op));
-        });
-      } else {
-        return Stream.empty();
-      }
+      return getPendingCompactionOperations(instantPlanPair.getKey(), instantPlanPair.getValue());
     }).forEach(pair -> {
       // Defensive check to ensure a single-fileId does not have more than one pending compaction
       if (fgIdToPendingCompactionWithInstantMap.containsKey(pair.getKey())) {
@@ -169,4 +159,17 @@ public class CompactionUtils {
     });
     return fgIdToPendingCompactionWithInstantMap;
   }
+
+  public static Stream<Pair<HoodieFileGroupId, Pair<String, HoodieCompactionOperation>>> getPendingCompactionOperations(
+      HoodieInstant instant, HoodieCompactionPlan compactionPlan) {
+    List<HoodieCompactionOperation> ops = compactionPlan.getOperations();
+    if (null != ops) {
+      return ops.stream().map(op -> {
+        return Pair.of(new HoodieFileGroupId(op.getPartitionPath(), op.getFileId()),
+            Pair.of(instant.getTimestamp(), op));
+      });
+    } else {
+      return Stream.empty();
+    }
+  }
 }
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/Functions.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/Functions.java
new file mode 100644
index 0000000..ca9bd13
--- /dev/null
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/Functions.java
@@ -0,0 +1,38 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *          http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.uber.hoodie.common.util;
+
+import java.io.Serializable;
+
+public interface Functions {
+
+  public interface Function0<R> extends Serializable {
+    R apply();
+  }
+
+  public interface Function1<T1, R> extends Serializable {
+    R apply(T1 val1);
+  }
+
+  public interface Function2<T1, T2, R> extends Serializable {
+    R apply(T1 val1, T2 val2);
+  }
+
+  public interface Function3<T1, T2, T3, R> extends Serializable {
+    R apply(T1 val1, T2 val2, T3 val3);
+  }
+}
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/NetworkUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/NetworkUtils.java
new file mode 100644
index 0000000..c071955
--- /dev/null
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/NetworkUtils.java
@@ -0,0 +1,42 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *          http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.uber.hoodie.common.util;
+
+import com.uber.hoodie.exception.HoodieException;
+import java.io.IOException;
+import java.net.ServerSocket;
+
+public class NetworkUtils {
+
+  public static synchronized String getHostname() {
+    ServerSocket s = null;
+    try {
+      s = new ServerSocket(0);
+      return s.getInetAddress().getHostAddress();
+    } catch (IOException e) {
+      throw new HoodieException("Unable to find server port", e);
+    } finally {
+      if (null != s) {
+        try {
+          s.close();
+        } catch (IOException e) {
+          throw new HoodieException("Unable to close server port", e);
+        }
+      }
+    }
+  }
+}
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/Option.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/Option.java
new file mode 100644
index 0000000..63e2ef1
--- /dev/null
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/Option.java
@@ -0,0 +1,312 @@
+/*
+ *  Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  You may obtain a copy of the License at
+ *
+ *           http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package com.uber.hoodie.common.util;
+
+import java.io.Serializable;
+import java.util.NoSuchElementException;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import java.util.function.Predicate;
+import java.util.function.Supplier;
+
+/**
+ * Copied from java.util.Optional and made Serializable along with methods to convert to/from standard Option
+ */
+public final class Option<T> implements Serializable {
+
+  private static final long serialVersionUID = 0L;
+
+  /**
+   * Common instance for {@code empty()}.
+   */
+  private static final Option<?> EMPTY = new Option<>();
+
+  /**
+   * If non-null, the value; if null, indicates no value is present
+   */
+  private final T value;
+
+  /**
+   * Constructs an empty instance.
+   *
+   * @implNote Generally only one empty instance, {@link Option#EMPTY}, should exist per VM.
+   */
+  private Option() {
+    this.value = null;
+  }
+
+  /**
+   * Returns an empty {@code Option} instance.  No value is present for this Option.
+   *
+   * @param <T> Type of the non-existent value
+   * @return an empty {@code Option}
+   * @apiNote Though it may be tempting to do so, avoid testing if an object is empty by comparing with {@code ==}
+   * against instances returned by {@code Option.empty()}. There is no guarantee that it is a singleton. Instead, use
+   * {@link #isPresent()}.
+   */
+  public static <T> Option<T> empty() {
+    @SuppressWarnings("unchecked")
+    Option<T> t = (Option<T>) EMPTY;
+    return t;
+  }
+
+  /**
+   * Constructs an instance with the value present.
+   *
+   * @param value the non-null value to be present
+   * @throws NullPointerException if value is null
+   */
+  private Option(T value) {
+    this.value = Objects.requireNonNull(value);
+  }
+
+  /**
+   * Returns an {@code Option} with the specified present non-null value.
+   *
+   * @param <T> the class of the value
+   * @param value the value to be present, which must be non-null
+   * @return an {@code Option} with the value present
+   * @throws NullPointerException if value is null
+   */
+  public static <T> Option<T> of(T value) {
+    return new Option<>(value);
+  }
+
+  /**
+   * Returns an {@code Option} describing the specified value, if non-null, otherwise returns an empty {@code Option}.
+   *
+   * @param <T> the class of the value
+   * @param value the possibly-null value to describe
+   * @return an {@code Option} with a present value if the specified value is non-null, otherwise an empty {@code
+   * Option}
+   */
+  public static <T> Option<T> ofNullable(T value) {
+    return value == null ? empty() : of(value);
+  }
+
+  /**
+   * If a value is present in this {@code Option}, returns the value, otherwise throws {@code NoSuchElementException}.
+   *
+   * @return the non-null value held by this {@code Option}
+   * @throws NoSuchElementException if there is no value present
+   * @see Option#isPresent()
+   */
+  public T get() {
+    if (value == null) {
+      throw new NoSuchElementException("No value present");
+    }
+    return value;
+  }
+
+  /**
+   * Return {@code true} if there is a value present, otherwise {@code false}.
+   *
+   * @return {@code true} if there is a value present, otherwise {@code false}
+   */
+  public boolean isPresent() {
+    return value != null;
+  }
+
+  /**
+   * If a value is present, invoke the specified consumer with the value, otherwise do nothing.
+   *
+   * @param consumer block to be executed if a value is present
+   * @throws NullPointerException if value is present and {@code consumer} is null
+   */
+  public void ifPresent(Consumer<? super T> consumer) {
+    if (value != null) {
+      consumer.accept(value);
+    }
+  }
+
+  /**
+   * If a value is present, and the value matches the given predicate, return an {@code Option} describing the value,
+   * otherwise return an empty {@code Option}.
+   *
+   * @param predicate a predicate to apply to the value, if present
+   * @return an {@code Option} describing the value of this {@code Option} if a value is present and the value matches
+   * the given predicate, otherwise an empty {@code Option}
+   * @throws NullPointerException if the predicate is null
+   */
+  public Option<T> filter(Predicate<? super T> predicate) {
+    Objects.requireNonNull(predicate);
+    if (!isPresent()) {
+      return this;
+    } else {
+      return predicate.test(value) ? this : empty();
+    }
+  }
+
+  /**
+   * If a value is present, apply the provided mapping function to it, and if the result is non-null, return an {@code
+   * Option} describing the result.  Otherwise return an empty {@code Option}.
+   *
+   * @param <U> The type of the result of the mapping function
+   * @param mapper a mapping function to apply to the value, if present
+   * @return an {@code Option} describing the result of applying a mapping function to the value of this {@code Option},
+   * if a value is present, otherwise an empty {@code Option}
+   * @throws NullPointerException if the mapping function is null
+   * @apiNote This method supports post-processing on optional values, without the need to explicitly check for a return
+   * status.  For example, the following code traverses a stream of file names, selects one that has not yet been
+   * processed, and then opens that file, returning an {@code Option<FileInputStream>}:
+   *
+   * <pre>{@code
+   *     Option<FileInputStream> fis =
+   *         names.stream().filter(name -> !isProcessedYet(name))
+   *                       .findFirst()
+   *                       .map(name -> new FileInputStream(name));
+   * }</pre>
+   *
+   * Here, {@code findFirst} returns an {@code Option<String>}, and then {@code map} returns an {@code
+   * Option<FileInputStream>} for the desired file if one exists.
+   */
+  public <U> Option<U> map(Function<? super T, ? extends U> mapper) {
+    Objects.requireNonNull(mapper);
+    if (!isPresent()) {
+      return empty();
+    } else {
+      return Option.ofNullable(mapper.apply(value));
+    }
+  }
+
+  /**
+   * If a value is present, apply the provided {@code Option}-bearing mapping function to it, return that result,
+   * otherwise return an empty {@code Option}.  This method is similar to {@link #map(Function)}, but the provided
+   * mapper is one whose result is already an {@code Option}, and if invoked, {@code flatMap} does not wrap it with an
+   * additional {@code Option}.
+   *
+   * @param <U> The type parameter to the {@code Option} returned by
+   * @param mapper a mapping function to apply to the value, if present the mapping function
+   * @return the result of applying an {@code Option}-bearing mapping function to the value of this {@code Option}, if a
+   * value is present, otherwise an empty {@code Option}
+   * @throws NullPointerException if the mapping function is null or returns a null result
+   */
+  public <U> Option<U> flatMap(Function<? super T, Option<U>> mapper) {
+    Objects.requireNonNull(mapper);
+    if (!isPresent()) {
+      return empty();
+    } else {
+      return Objects.requireNonNull(mapper.apply(value));
+    }
+  }
+
+  /**
+   * Return the value if present, otherwise return {@code other}.
+   *
+   * @param other the value to be returned if there is no value present, may be null
+   * @return the value, if present, otherwise {@code other}
+   */
+  public T orElse(T other) {
+    return value != null ? value : other;
+  }
+
+  /**
+   * Return the value if present, otherwise invoke {@code other} and return the result of that invocation.
+   *
+   * @param other a {@code Supplier} whose result is returned if no value is present
+   * @return the value if present otherwise the result of {@code other.get()}
+   * @throws NullPointerException if value is not present and {@code other} is null
+   */
+  public T orElseGet(Supplier<? extends T> other) {
+    return value != null ? value : other.get();
+  }
+
+  /**
+   * Return the contained value, if present, otherwise throw an exception to be created by the provided supplier.
+   *
+   * @param <X> Type of the exception to be thrown
+   * @param exceptionSupplier The supplier which will return the exception to be thrown
+   * @return the present value
+   * @throws X if there is no value present
+   * @throws NullPointerException if no value is present and {@code exceptionSupplier} is null
+   * @apiNote A method reference to the exception constructor with an empty argument list can be used as the supplier.
+   * For example, {@code IllegalStateException::new}
+   */
+  public <X extends Throwable> T orElseThrow(Supplier<? extends X> exceptionSupplier) throws X {
+    if (value != null) {
+      return value;
+    } else {
+      throw exceptionSupplier.get();
+    }
+  }
+
+  /**
+   * Indicates whether some other object is "equal to" this Option. The other object is considered equal if:
+   * <ul>
+   * <li>it is also an {@code Option} and;
+   * <li>both instances have no value present or;
+   * <li>the present values are "equal to" each other via {@code equals()}.
+   * </ul>
+   *
+   * @param obj an object to be tested for equality
+   * @return {code true} if the other object is "equal to" this object otherwise {@code false}
+   */
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+
+    if (!(obj instanceof Option)) {
+      return false;
+    }
+
+    Option<?> other = (Option<?>) obj;
+    return Objects.equals(value, other.value);
+  }
+
+  /**
+   * Returns the hash code value of the present value, if any, or 0 (zero) if no value is present.
+   *
+   * @return hash code value of the present value or 0 if no value is present
+   */
+  @Override
+  public int hashCode() {
+    return Objects.hashCode(value);
+  }
+
+  /**
+   * Returns a non-empty string representation of this Option suitable for debugging. The exact presentation format is
+   * unspecified and may vary between implementations and versions.
+   *
+   * @return the string representation of this instance
+   * @implSpec If a value is present the result must include its string representation in the result. Empty and present
+   * Optionals must be unambiguously differentiable.
+   */
+  @Override
+  public String toString() {
+    return value != null
+        ? String.format("Option[%s]", value)
+        : "Option.empty";
+  }
+
+  /**
+   * Convert to java Optional
+   */
+  public Optional<T> toJavaOptional() {
+    return Optional.ofNullable(value);
+  }
+
+  /**
+   * Convert from java.util.Optional
+   */
+  public static <T> Option<T> fromJavaOptional(Optional<T> v) {
+    return Option.ofNullable(v.orElse(null));
+  }
+}
\ No newline at end of file
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/RocksDBDAO.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/RocksDBDAO.java
new file mode 100644
index 0000000..29cd8d6
--- /dev/null
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/RocksDBDAO.java
@@ -0,0 +1,389 @@
+/*
+ *  Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  You may obtain a copy of the License at
+ *
+ *           http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package com.uber.hoodie.common.util;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.uber.hoodie.common.table.view.FileSystemViewStorageConfig;
+import com.uber.hoodie.common.util.collection.Pair;
+import com.uber.hoodie.exception.HoodieException;
+import com.uber.hoodie.exception.HoodieIOException;
+import java.io.File;
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.commons.io.FileUtils;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.InfoLogLevel;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+import org.rocksdb.RocksIterator;
+import org.rocksdb.Statistics;
+import org.rocksdb.WriteBatch;
+import org.rocksdb.WriteOptions;
+
+/**
+ * Data access objects for storing and retrieving objects in Rocks DB.
+ */
+public class RocksDBDAO {
+
+  protected static final transient Logger log = LogManager.getLogger(RocksDBDAO.class);
+
+  private final FileSystemViewStorageConfig config;
+  private transient ConcurrentHashMap<String, ColumnFamilyHandle> managedHandlesMap;
+  private transient ConcurrentHashMap<String, ColumnFamilyDescriptor> managedDescriptorMap;
+  private transient RocksDB rocksDB;
+  private boolean closed = false;
+  private final String basePath;
+  private final String rocksDBBasePath;
+
+  public RocksDBDAO(String basePath, FileSystemViewStorageConfig config) {
+    this.basePath = basePath;
+    this.config = config;
+    this.rocksDBBasePath = String.format("%s/%s/%s", config.getRocksdbBasePath(),
+        this.basePath.replace("/", "_"), UUID.randomUUID().toString());
+    init();
+  }
+
+  /**
+   * Create RocksDB if not initialized.
+   */
+  private RocksDB getRocksDB() {
+    if (null == rocksDB) {
+      init();
+    }
+    return rocksDB;
+  }
+
+  /**
+   * Initialized Rocks DB instance
+   */
+  private void init() throws HoodieException {
+    try {
+      log.warn("DELETING RocksDB persisted at " + rocksDBBasePath);
+      FileUtils.deleteDirectory(new File(rocksDBBasePath));
+
+      managedHandlesMap = new ConcurrentHashMap<>();
+      managedDescriptorMap = new ConcurrentHashMap<>();
+
+      // If already present, loads the existing column-family handles
+      final DBOptions dbOptions = new DBOptions().setCreateIfMissing(true).setCreateMissingColumnFamilies(true)
+          .setWalDir(rocksDBBasePath).setStatsDumpPeriodSec(300).setStatistics(new Statistics());
+      dbOptions.setLogger(new org.rocksdb.Logger(dbOptions) {
+        @Override
+        protected void log(InfoLogLevel infoLogLevel, String logMsg) {
+          log.info("From Rocks DB : " + logMsg);
+        }
+      });
+      final List<ColumnFamilyDescriptor> managedColumnFamilies = loadManagedColumnFamilies(dbOptions);
+      final List<ColumnFamilyHandle> managedHandles = new ArrayList<>();
+      FileUtils.forceMkdir(new File(rocksDBBasePath));
+      rocksDB = RocksDB.open(dbOptions, rocksDBBasePath, managedColumnFamilies, managedHandles);
+
+      Preconditions.checkArgument(managedHandles.size() == managedColumnFamilies.size(),
+          "Unexpected number of handles are returned");
+      for (int index = 0; index < managedHandles.size(); index++) {
+        ColumnFamilyHandle handle = managedHandles.get(index);
+        ColumnFamilyDescriptor descriptor = managedColumnFamilies.get(index);
+        String familyNameFromHandle = new String(handle.getName());
+        String familyNameFromDescriptor = new String(descriptor.getName());
+
+        Preconditions.checkArgument(familyNameFromDescriptor.equals(familyNameFromHandle),
+            "Family Handles not in order with descriptors");
+        managedHandlesMap.put(familyNameFromHandle, handle);
+        managedDescriptorMap.put(familyNameFromDescriptor, descriptor);
+      }
+    } catch (RocksDBException | IOException re) {
+      log.error("Got exception opening rocks db instance ", re);
+      throw new HoodieException(re);
+    }
+  }
+
+  /**
+   * Helper to load managed column family descriptors
+   */
+  private List<ColumnFamilyDescriptor> loadManagedColumnFamilies(DBOptions dbOptions) throws RocksDBException {
+    final List<ColumnFamilyDescriptor> managedColumnFamilies = new ArrayList<>();
+    final Options options = new Options(dbOptions, new ColumnFamilyOptions());
+    List<byte[]> existing = RocksDB.listColumnFamilies(options, rocksDBBasePath);
+
+    if (existing.isEmpty()) {
+      log.info("No column family found. Loading default");
+      managedColumnFamilies.add(getColumnFamilyDescriptor(RocksDB.DEFAULT_COLUMN_FAMILY));
+    } else {
+      log.info("Loading column families :" + existing.stream().map(String::new).collect(Collectors.toList()));
+      managedColumnFamilies.addAll(existing.stream()
+          .map(RocksDBDAO::getColumnFamilyDescriptor).collect(Collectors.toList()));
+    }
+    return managedColumnFamilies;
+  }
+
+  private static ColumnFamilyDescriptor getColumnFamilyDescriptor(byte[] columnFamilyName) {
+    return new ColumnFamilyDescriptor(columnFamilyName, new ColumnFamilyOptions());
+  }
+
+  /**
+   * Perform a batch write operation
+   */
+  public void writeBatch(BatchHandler handler) {
+    WriteBatch batch = new WriteBatch();
+    try {
+      handler.apply(batch);
+      getRocksDB().write(new WriteOptions(), batch);
+    } catch (RocksDBException re) {
+      throw new HoodieException(re);
+    } finally {
+      batch.close();
+    }
+  }
+
+  /**
+   * Helper to add put operation in batch
+   *
+   * @param batch Batch Handle
+   * @param columnFamilyName Column Family
+   * @param key Key
+   * @param value Payload
+   * @param <T> Type of payload
+   */
+  public <T extends Serializable> void putInBatch(WriteBatch batch, String columnFamilyName, String key, T value) {
+    try {
+      byte[] payload = SerializationUtils.serialize(value);
+      batch.put(managedHandlesMap.get(columnFamilyName), key.getBytes(), payload);
+    } catch (Exception e) {
+      throw new HoodieException(e);
+    }
+  }
+
+  /**
+   * Perform single PUT on a column-family
+   *
+   * @param columnFamilyName Column family name
+   * @param key Key
+   * @param value Payload
+   * @param <T> Type of Payload
+   */
+  public <T extends Serializable> void put(String columnFamilyName, String key, T value) {
+    try {
+      byte[] payload = SerializationUtils.serialize(value);
+      getRocksDB().put(managedHandlesMap.get(columnFamilyName), key.getBytes(), payload);
+    } catch (Exception e) {
+      throw new HoodieException(e);
+    }
+  }
+
+  /**
+   * Helper to add delete operation in batch
+   *
+   * @param batch Batch Handle
+   * @param columnFamilyName Column Family
+   * @param key Key
+   */
+  public void deleteInBatch(WriteBatch batch, String columnFamilyName, String key) {
+    try {
+      batch.delete(managedHandlesMap.get(columnFamilyName), key.getBytes());
+    } catch (RocksDBException e) {
+      throw new HoodieException(e);
+    }
+  }
+
+  /**
+   * Perform a single Delete operation
+   *
+   * @param columnFamilyName Column Family name
+   * @param key Key to be deleted
+   */
+  public void delete(String columnFamilyName, String key) {
+    try {
+      getRocksDB().delete(managedHandlesMap.get(columnFamilyName), key.getBytes());
+    } catch (RocksDBException e) {
+      throw new HoodieException(e);
+    }
+  }
+
+  /**
+   * Retrieve a value for a given key in a column family
+   *
+   * @param columnFamilyName Column Family Name
+   * @param key Key to be retrieved
+   * @param <T> Type of object stored.
+   */
+  public <T extends Serializable> T get(String columnFamilyName, String key) {
+    Preconditions.checkArgument(!closed);
+    try {
+      byte[] val = getRocksDB().get(managedHandlesMap.get(columnFamilyName), key.getBytes());
+      return val == null ? null : SerializationUtils.deserialize(val);
+    } catch (RocksDBException e) {
+      throw new HoodieException(e);
+    }
+  }
+
+  /**
+   * Perform a prefix search and return stream of key-value pairs retrieved
+   *
+   * @param columnFamilyName Column Family Name
+   * @param prefix Prefix Key
+   * @param <T> Type of value stored
+   */
+  public <T extends Serializable> Stream<Pair<String, T>> prefixSearch(String columnFamilyName, String prefix) {
+    Preconditions.checkArgument(!closed);
+    final HoodieTimer timer = new HoodieTimer();
+    timer.startTimer();
+    long timeTakenMicro = 0;
+    List<Pair<String, T>> results = new LinkedList<>();
+    try (final RocksIterator it = getRocksDB().newIterator(managedHandlesMap.get(columnFamilyName))) {
+      it.seek(prefix.getBytes());
+      while (it.isValid() && new String(it.key()).startsWith(prefix)) {
+        long beginTs = System.nanoTime();
+        T val = SerializationUtils.deserialize(it.value());
+        timeTakenMicro += ((System.nanoTime() - beginTs) / 1000);
+        results.add(Pair.of(new String(it.key()), val));
+        it.next();
+      }
+    }
+
+    log.info("Prefix Search for (query=" + prefix + ") on " + columnFamilyName
+        + ". Total Time Taken (msec)=" + timer.endTimer()
+        + ". Serialization Time taken(micro)=" + timeTakenMicro + ", num entries=" + results.size());
+    return results.stream();
+  }
+
+  /**
+   * Perform a prefix delete and return stream of key-value pairs retrieved
+   *
+   * @param columnFamilyName Column Family Name
+   * @param prefix Prefix Key
+   * @param <T> Type of value stored
+   */
+  public <T extends Serializable> void prefixDelete(String columnFamilyName, String prefix) {
+    Preconditions.checkArgument(!closed);
+    log.info("Prefix DELETE (query=" + prefix + ") on " + columnFamilyName);
+    final RocksIterator it = getRocksDB().newIterator(managedHandlesMap.get(columnFamilyName));
+    it.seek(prefix.getBytes());
+    //Find first and last keys to be deleted
+    String firstEntry = null;
+    String lastEntry = null;
+    while (it.isValid() && new String(it.key()).startsWith(prefix)) {
+      String result = new String(it.key());
+      it.next();
+      if (firstEntry == null) {
+        firstEntry = result;
+      }
+      lastEntry = result;
+    }
+    it.close();
+
+    if (null != firstEntry) {
+      try {
+        // This will not delete the last entry
+        getRocksDB().deleteRange(managedHandlesMap.get(columnFamilyName), firstEntry.getBytes(),
+            lastEntry.getBytes());
+        //Delete the last entry
+        getRocksDB().delete(lastEntry.getBytes());
+      } catch (RocksDBException e) {
+        log.error("Got exception performing range delete");
+        throw new HoodieException(e);
+      }
+    }
+  }
+
+  /**
+   * Add a new column family to store
+   *
+   * @param columnFamilyName Column family name
+   */
+  public void addColumnFamily(String columnFamilyName) {
+    Preconditions.checkArgument(!closed);
+
+    managedDescriptorMap.computeIfAbsent(columnFamilyName, colFamilyName -> {
+      try {
+        ColumnFamilyDescriptor descriptor = getColumnFamilyDescriptor(colFamilyName.getBytes());
+        ColumnFamilyHandle handle = getRocksDB().createColumnFamily(descriptor);
+        managedHandlesMap.put(colFamilyName, handle);
+        return descriptor;
+      } catch (RocksDBException e) {
+        throw new HoodieException(e);
+      }
+    });
+  }
+
+  /**
+   * Note : Does not delete from underlying DB. Just closes the handle
+   *
+   * @param columnFamilyName Column Family Name
+   */
+  public void dropColumnFamily(String columnFamilyName) {
+    Preconditions.checkArgument(!closed);
+
+    managedDescriptorMap.computeIfPresent(columnFamilyName, (colFamilyName, descriptor) -> {
+      ColumnFamilyHandle handle = managedHandlesMap.get(colFamilyName);
+      try {
+        getRocksDB().dropColumnFamily(handle);
+        handle.close();
+      } catch (RocksDBException e) {
+        throw new HoodieException(e);
+      }
+      managedHandlesMap.remove(columnFamilyName);
+      return null;
+    });
+  }
+
+  /**
+   * Close the DAO object
+   */
+  public synchronized void close() {
+    if (!closed) {
+      closed = true;
+      managedHandlesMap.values().forEach(columnFamilyHandle -> {
+        columnFamilyHandle.close();
+      });
+      managedHandlesMap.clear();
+      managedDescriptorMap.clear();
+      getRocksDB().close();
+      try {
+        FileUtils.deleteDirectory(new File(rocksDBBasePath));
+      } catch (IOException e) {
+        throw new HoodieIOException(e.getMessage(), e);
+      }
+    }
+  }
+
+  @VisibleForTesting
+  String getRocksDBBasePath() {
+    return rocksDBBasePath;
+  }
+
+  /**
+   * Functional interface for stacking operation to Write batch
+   */
+  public interface BatchHandler {
+
+    void apply(WriteBatch batch);
+  }
+}
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/RocksDBSchemaHelper.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/RocksDBSchemaHelper.java
new file mode 100644
index 0000000..ed8a88a
--- /dev/null
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/RocksDBSchemaHelper.java
@@ -0,0 +1,124 @@
+/*
+ *  Copyright (c) 2019 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  You may obtain a copy of the License at
+ *
+ *           http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package com.uber.hoodie.common.util;
+
+import com.uber.hoodie.common.model.FileSlice;
+import com.uber.hoodie.common.model.HoodieFileGroup;
+import com.uber.hoodie.common.model.HoodieFileGroupId;
+import com.uber.hoodie.common.table.HoodieTableMetaClient;
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * Helper class to generate Key and column names for rocksdb based view
+ *
+ * For RocksDB, 3 colFamilies are used for storing file-system view for each dataset.
+ *   (a) View
+ *   (b) Partitions Cached
+ *   (c) Pending Compactions
+ *
+ *
+ * View : Key : Store both slice and Data file stored.
+ *     Slice :
+ *         Key   = "type=slice,part=<PartitionPath>,id=<FileId>,instant=<Timestamp>"
+ *         Value = Serialized FileSlice
+ *     Data File :
+ *         Key   = "type=df,part=<PartitionPath>,id=<FileId>,instant=<Timestamp>"
+ *         Value =  Serialized DataFile
+ *
+ * Partitions :
+ *       Key  = "part=<PartitionPath>"
+ *       Value = Boolean
+ *
+ * Pending Compactions
+ *       Key = "part=<PartitionPath>,id=<FileId>"
+ *       Value = Pair<CompactionTime, CompactionOperation>
+ */
+public class RocksDBSchemaHelper {
+
+  private final String colFamilyForView;
+  private final String colFamilyForPendingCompaction;
+  private final String colFamilyForStoredPartitions;
+
+  public RocksDBSchemaHelper(HoodieTableMetaClient metaClient) {
+    this.colFamilyForPendingCompaction = "hudi_pending_compaction_" + metaClient.getBasePath().replace("/", "_");
+    this.colFamilyForStoredPartitions = "hudi_partitions_" + metaClient.getBasePath().replace("/", "_");
+    this.colFamilyForView = "hudi_view_" + metaClient.getBasePath().replace("/", "_");
+  }
+
+  public List<String> getAllColumnFamilies() {
+    return Arrays.asList(getColFamilyForView(), getColFamilyForPendingCompaction(), getColFamilyForStoredPartitions());
+  }
+
+  public String getKeyForPartitionLookup(String partition) {
+    return String.format("part=%s", partition);
+  }
+
+  public String getKeyForPendingCompactionLookup(HoodieFileGroupId fgId) {
+    return getPartitionFileIdBasedLookup(fgId);
+  }
+
+  public String getKeyForSliceView(HoodieFileGroup fileGroup, FileSlice slice) {
+    return getKeyForSliceView(fileGroup.getPartitionPath(), fileGroup.getFileGroupId().getFileId(),
+        slice.getBaseInstantTime());
+  }
+
+  public String getKeyForSliceView(String partitionPath, String fileId, String instantTime) {
+    return String.format("type=slice,part=%s,id=%s,instant=%s",partitionPath, fileId, instantTime);
+  }
+
+  public String getPrefixForSliceViewByPartitionFile(String partitionPath, String fileId) {
+    return String.format("type=slice,part=%s,id=%s,instant=",partitionPath, fileId);
+  }
+
+  public String getPrefixForDataFileViewByPartitionFile(String partitionPath, String fileId) {
+    return String.format("type=df,part=%s,id=%s,instant=",partitionPath, fileId);
+  }
+
+  public String getKeyForDataFileView(HoodieFileGroup fileGroup, FileSlice slice) {
+    return String.format("type=df,part=%s,id=%s,instant=%s", fileGroup.getPartitionPath(),
+        fileGroup.getFileGroupId().getFileId(), slice.getBaseInstantTime());
+  }
+
+  public String getPrefixForSliceViewByPartition(String partitionPath) {
+    return String.format("type=slice,part=%s,id=", partitionPath);
+  }
+
+  public String getPrefixForSliceView() {
+    return "type=slice,part=";
+  }
+
+  public String getPrefixForDataFileViewByPartition(String partitionPath) {
+    return String.format("type=df,part=%s,id=", partitionPath);
+  }
+
+  private String getPartitionFileIdBasedLookup(HoodieFileGroupId fgId) {
+    return String.format("part=%s,id=%s", fgId.getPartitionPath(), fgId.getFileId());
+  }
+
+  public String getColFamilyForView() {
+    return colFamilyForView;
+  }
+
+  public String getColFamilyForPendingCompaction() {
+    return colFamilyForPendingCompaction;
+  }
+
+  public String getColFamilyForStoredPartitions() {
+    return colFamilyForStoredPartitions;
+  }
+}
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/StringUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/StringUtils.java
index e9921a2..80465eb 100644
--- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/StringUtils.java
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/StringUtils.java
@@ -41,6 +41,10 @@ public class StringUtils {
     return join(elements, "");
   }
 
+  public static <T> String joinUsingDelim(String delim, final String... elements) {
+    return join(elements, delim);
+  }
+
   public static String join(final String[] array, final String separator) {
     if (array == null) {
       return null;
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/TimelineDiffHelper.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/TimelineDiffHelper.java
new file mode 100644
index 0000000..73e0d37
--- /dev/null
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/TimelineDiffHelper.java
@@ -0,0 +1,134 @@
+/*
+ * Copyright (c) 2019 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *          http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.uber.hoodie.common.util;
+
+import com.uber.hoodie.common.table.HoodieTimeline;
+import com.uber.hoodie.common.table.timeline.HoodieInstant;
+import com.uber.hoodie.common.table.timeline.HoodieInstant.State;
+import com.uber.hoodie.common.util.collection.Pair;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+
+public class TimelineDiffHelper {
+
+  protected static Logger log = LogManager.getLogger(TimelineDiffHelper.class);
+
+  public static TimelineDiffResult getNewInstantsForIncrementalSync(HoodieTimeline oldTimeline,
+      HoodieTimeline newTimeline) {
+
+    HoodieTimeline oldT = oldTimeline.filterCompletedAndCompactionInstants();
+    HoodieTimeline newT = newTimeline.filterCompletedAndCompactionInstants();
+
+    Optional<HoodieInstant> lastSeenInstant = oldT.lastInstant();
+    Optional<HoodieInstant> firstInstantInNewTimeline = newT.firstInstant();
+
+    if (lastSeenInstant.isPresent() && firstInstantInNewTimeline.isPresent()) {
+      if (HoodieTimeline.compareTimestamps(lastSeenInstant.get().getTimestamp(),
+          firstInstantInNewTimeline.get().getTimestamp(), HoodieTimeline.LESSER)) {
+        // The last seen instant is no longer in the timeline. Do not incrementally Sync.
+        return TimelineDiffResult.UNSAFE_SYNC_RESULT;
+      }
+      Set<HoodieInstant> oldTimelineInstants = oldT.getInstants().collect(Collectors.toSet());
+
+      List<HoodieInstant> newInstants = new ArrayList<>();
+
+      // Check If any pending compaction is lost. If so, do not allow incremental timeline sync
+      List<Pair<HoodieInstant, HoodieInstant>> compactionInstants = getPendingCompactionTransitions(oldT, newT);
+      List<HoodieInstant> lostPendingCompactions =
+          compactionInstants.stream().filter(instantPair -> instantPair.getValue() == null).map(Pair::getKey)
+              .collect(Collectors.toList());
+      if (!lostPendingCompactions.isEmpty()) {
+        // If a compaction is unscheduled, fall back to complete refresh of fs view since some log files could have been
+        // moved. Its unsafe to incrementally sync in that case.
+        log.warn("Some pending compactions are no longer in new timeline (unscheduled ?)."
+            + "They are :" + lostPendingCompactions);
+        return TimelineDiffResult.UNSAFE_SYNC_RESULT;
+      }
+      List<HoodieInstant> finishedCompactionInstants = compactionInstants.stream().filter(instantPair ->
+          instantPair.getValue().getAction().equals(HoodieTimeline.COMMIT_ACTION)
+              && instantPair.getValue().isCompleted()).map(Pair::getKey).collect(Collectors.toList());
+
+      newT.getInstants().filter(instant -> !oldTimelineInstants.contains(instant)).forEach(newInstants::add);
+      return new TimelineDiffResult(newInstants, finishedCompactionInstants, true);
+    } else {
+      // One or more timelines is empty
+      log.warn("One or more timelines is empty");
+      return TimelineDiffResult.UNSAFE_SYNC_RESULT;
+    }
+  }
+
+  private static List<Pair<HoodieInstant, HoodieInstant>> getPendingCompactionTransitions(HoodieTimeline oldTimeline,
+      HoodieTimeline newTimeline) {
+    Set<HoodieInstant> newTimelineInstants = newTimeline.getInstants().collect(Collectors.toSet());
+
+    return oldTimeline.filterPendingCompactionTimeline().getInstants().map(instant -> {
+      if (newTimelineInstants.contains(instant)) {
+        return Pair.of(instant, instant);
+      } else {
+        HoodieInstant compacted =
+            new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, instant.getTimestamp());
+        if (newTimelineInstants.contains(compacted)) {
+          return Pair.of(instant, compacted);
+        }
+        return Pair.<HoodieInstant, HoodieInstant>of(instant, null);
+      }
+    }).collect(Collectors.toList());
+  }
+
+  public static class TimelineDiffResult {
+
+    private final List<HoodieInstant> newlySeenInstants;
+    private final List<HoodieInstant> finishedCompactionInstants;
+    private final boolean canSyncIncrementally;
+
+    public static final TimelineDiffResult UNSAFE_SYNC_RESULT = new TimelineDiffResult(null, null, false);
+
+    public TimelineDiffResult(List<HoodieInstant> newlySeenInstants, List<HoodieInstant> finishedCompactionInstants,
+        boolean canSyncIncrementally) {
+      this.newlySeenInstants = newlySeenInstants;
+      this.finishedCompactionInstants = finishedCompactionInstants;
+      this.canSyncIncrementally = canSyncIncrementally;
+    }
+
+    public List<HoodieInstant> getNewlySeenInstants() {
+      return newlySeenInstants;
+    }
+
+    public List<HoodieInstant> getFinishedCompactionInstants() {
+      return finishedCompactionInstants;
+    }
+
+    public boolean canSyncIncrementally() {
+      return canSyncIncrementally;
+    }
+
+    @Override
+    public String toString() {
+      return "TimelineDiffResult{"
+          + "newlySeenInstants=" + newlySeenInstants
+          + ", finishedCompactionInstants=" + finishedCompactionInstants
+          + ", canSyncIncrementally=" + canSyncIncrementally
+          + '}';
+    }
+  }
+}
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/DiskBasedMap.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/DiskBasedMap.java
index 9fd0091..85b65c4 100644
--- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/DiskBasedMap.java
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/DiskBasedMap.java
@@ -31,13 +31,16 @@ import java.net.InetAddress;
 import java.util.AbstractMap;
 import java.util.Collection;
 import java.util.Date;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.Map;
+import java.util.Queue;
 import java.util.Set;
 import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.atomic.AtomicLong;
+import java.util.stream.Stream;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
 
@@ -51,8 +54,6 @@ public final class DiskBasedMap<T extends Serializable, R extends Serializable>
   private static final Logger log = LogManager.getLogger(DiskBasedMap.class);
   // Stores the key and corresponding value's latest metadata spilled to disk
   private final Map<T, ValueMetadata> valueMetadataMap;
-  // Read only file access to be able to seek to random positions to readFromDisk values
-  private RandomAccessFile readOnlyFileHandle;
   // Write only OutputStream to be able to ONLY append to the file
   private SizeAwareDataOutputStream writeOnlyFileHandle;
   // FileOutputStream for the file handle to be able to force fsync
@@ -62,10 +63,12 @@ public final class DiskBasedMap<T extends Serializable, R extends Serializable>
   private AtomicLong filePosition;
   // FilePath to store the spilled data
   private String filePath;
+  // Thread-safe random access file
+  private ThreadLocal<RandomAccessFile> randomAccessFile = new ThreadLocal<>();
+  private Queue<RandomAccessFile> openedAccessFiles = new ConcurrentLinkedQueue<>();
 
-
-  protected DiskBasedMap(String baseFilePath) throws IOException {
-    this.valueMetadataMap = new HashMap<>();
+  public DiskBasedMap(String baseFilePath) throws IOException {
+    this.valueMetadataMap = new ConcurrentHashMap<>();
     File writeOnlyFileHandle = new File(baseFilePath, UUID.randomUUID().toString());
     this.filePath = writeOnlyFileHandle.getPath();
     initFile(writeOnlyFileHandle);
@@ -74,6 +77,25 @@ public final class DiskBasedMap<T extends Serializable, R extends Serializable>
     this.filePosition = new AtomicLong(0L);
   }
 
+  /**
+   * RandomAcessFile is not thread-safe. This API opens a new file handle per thread and returns.
+   * @return
+   */
+  private RandomAccessFile getRandomAccessFile()  {
+    try {
+      RandomAccessFile readHandle = randomAccessFile.get();
+      if (readHandle == null) {
+        readHandle = new RandomAccessFile(filePath, "r");
+        readHandle.seek(0);
+        randomAccessFile.set(readHandle);
+        openedAccessFiles.offer(readHandle);
+      }
+      return readHandle;
+    } catch (IOException ioe) {
+      throw new HoodieException(ioe);
+    }
+  }
+
   private void initFile(File writeOnlyFileHandle) throws IOException {
     // delete the file if it exists
     if (writeOnlyFileHandle.exists()) {
@@ -86,9 +108,6 @@ public final class DiskBasedMap<T extends Serializable, R extends Serializable>
     log.info(
         "Spilling to file location " + writeOnlyFileHandle.getAbsolutePath() + " in host (" + InetAddress.getLocalHost()
             .getHostAddress() + ") with hostname (" + InetAddress.getLocalHost().getHostName() + ")");
-    // Open file in readFromDisk-only mode
-    readOnlyFileHandle = new RandomAccessFile(filePath, "r");
-    readOnlyFileHandle.seek(0);
     // Make sure file is deleted when JVM exits
     writeOnlyFileHandle.deleteOnExit();
     addShutDownHook();
@@ -107,6 +126,17 @@ public final class DiskBasedMap<T extends Serializable, R extends Serializable>
             fileOutputStream.getChannel().force(false);
             writeOnlyFileHandle.close();
           }
+
+          while (!openedAccessFiles.isEmpty()) {
+            RandomAccessFile file = openedAccessFiles.poll();
+            if (null != file) {
+              try {
+                file.close();
+              } catch (IOException ioe) {
+                // skip exception
+              }
+            }
+          }
         } catch (Exception e) {
           // fail silently for any sort of exception
         }
@@ -118,8 +148,7 @@ public final class DiskBasedMap<T extends Serializable, R extends Serializable>
    * Custom iterator to iterate over values written to disk
    */
   public Iterator<R> iterator() {
-    return new LazyFileIterable(readOnlyFileHandle,
-        valueMetadataMap).iterator();
+    return new LazyFileIterable(filePath, valueMetadataMap).iterator();
   }
 
   /**
@@ -155,16 +184,24 @@ public final class DiskBasedMap<T extends Serializable, R extends Serializable>
     if (entry == null) {
       return null;
     }
+    return get(entry);
+  }
+
+  private R get(ValueMetadata entry) {
+    return get(entry, getRandomAccessFile());
+  }
+
+  public static <R> R get(ValueMetadata entry, RandomAccessFile file) {
     try {
-      return SerializationUtils.<R>deserialize(SpillableMapUtils
-          .readBytesFromDisk(readOnlyFileHandle, entry.getOffsetOfValue(), entry.getSizeOfValue()));
+      return SerializationUtils.deserialize(SpillableMapUtils.readBytesFromDisk(file,
+          entry.getOffsetOfValue(), entry.getSizeOfValue()));
     } catch (IOException e) {
       throw new HoodieIOException("Unable to readFromDisk Hoodie Record from disk", e);
     }
   }
 
   @Override
-  public R put(T key, R value) {
+  public synchronized R put(T key, R value) {
     try {
       byte[] val = SerializationUtils.serialize(value);
       Integer valueSize = val.length;
@@ -198,14 +235,8 @@ public final class DiskBasedMap<T extends Serializable, R extends Serializable>
   @Override
   public void clear() {
     valueMetadataMap.clear();
-    // close input/output streams
-    try {
-      writeOnlyFileHandle.flush();
-      writeOnlyFileHandle.close();
-      new File(filePath).delete();
-    } catch (IOException e) {
-      throw new HoodieIOException("unable to clear map or delete file on disk", e);
-    }
+    // Do not delete file-handles & file as there is no way to do it without synchronizing get/put(and
+    // reducing concurrency). Instead, just clear the pointer map. The file will be removed on exit.
   }
 
   @Override
@@ -218,6 +249,12 @@ public final class DiskBasedMap<T extends Serializable, R extends Serializable>
     throw new HoodieException("Unsupported Operation Exception");
   }
 
+  public Stream<R> valueStream() {
+    final RandomAccessFile file = getRandomAccessFile();
+    return valueMetadataMap.values().stream().sorted().sequential()
+        .map(valueMetaData -> (R)get(valueMetaData, file));
+  }
+
   @Override
   public Set<Entry<T, R>> entrySet() {
     Set<Entry<T, R>> entrySet = new HashSet<>();
@@ -277,7 +314,7 @@ public final class DiskBasedMap<T extends Serializable, R extends Serializable>
     }
   }
 
-  public final class ValueMetadata {
+  public static final class ValueMetadata implements Comparable<ValueMetadata> {
 
     // FilePath to store the spilled data
     private String filePath;
@@ -310,5 +347,10 @@ public final class DiskBasedMap<T extends Serializable, R extends Serializable>
     public long getTimestamp() {
       return timestamp;
     }
+
+    @Override
+    public int compareTo(ValueMetadata o) {
+      return Long.compare(this.offsetOfValue, o.offsetOfValue);
+    }
   }
-}
\ No newline at end of file
+}
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/ExternalSpillableMap.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/ExternalSpillableMap.java
index 5ac94d7..ab95872 100644
--- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/ExternalSpillableMap.java
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/ExternalSpillableMap.java
@@ -18,15 +18,18 @@ package com.uber.hoodie.common.util.collection;
 
 import com.twitter.common.objectsize.ObjectSizeCalculator;
 import com.uber.hoodie.common.util.SizeEstimator;
-import com.uber.hoodie.exception.HoodieNotSupportedException;
+import com.uber.hoodie.exception.HoodieIOException;
 import java.io.IOException;
 import java.io.Serializable;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.stream.Stream;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
 
@@ -49,7 +52,7 @@ public class ExternalSpillableMap<T extends Serializable, R extends Serializable
   // Map to store key-values in memory until it hits maxInMemorySizeInBytes
   private final Map<T, R> inMemoryMap;
   // Map to store key-valuemetadata important to find the values spilled to disk
-  private final DiskBasedMap<T, R> diskBasedMap;
+  private transient volatile DiskBasedMap<T, R> diskBasedMap;
   // TODO(na) : a dynamic sizing factor to ensure we have space for other objects in memory and
   // incorrect payload estimation
   private final Double sizingFactorForInMemoryMap = 0.8;
@@ -63,10 +66,13 @@ public class ExternalSpillableMap<T extends Serializable, R extends Serializable
   private volatile long estimatedPayloadSize = 0;
   // Flag to determine whether to stop re-estimating payload size
   private boolean shouldEstimatePayloadSize = true;
-
+  // Base File Path
+  private final String baseFilePath;
+  
   public ExternalSpillableMap(Long maxInMemorySizeInBytes, String baseFilePath,
       SizeEstimator<T> keySizeEstimator, SizeEstimator<R> valueSizeEstimator) throws IOException {
     this.inMemoryMap = new HashMap<>();
+    this.baseFilePath = baseFilePath;
     this.diskBasedMap = new DiskBasedMap<>(baseFilePath);
     this.maxInMemorySizeInBytes = (long) Math
         .floor(maxInMemorySizeInBytes * sizingFactorForInMemoryMap);
@@ -75,25 +81,40 @@ public class ExternalSpillableMap<T extends Serializable, R extends Serializable
     this.valueSizeEstimator = valueSizeEstimator;
   }
 
+  private DiskBasedMap<T, R> getDiskBasedMap() {
+    if (null == diskBasedMap) {
+      synchronized (this) {
+        if (null == diskBasedMap) {
+          try {
+            diskBasedMap = new DiskBasedMap<>(baseFilePath);
+          } catch (IOException e) {
+            throw new HoodieIOException(e.getMessage(), e);
+          }
+        }
+      }
+    }
+    return diskBasedMap;
+  }
+
   /**
    * A custom iterator to wrap over iterating in-memory + disk spilled data
    */
   public Iterator<R> iterator() {
-    return new IteratorWrapper<>(inMemoryMap.values().iterator(), diskBasedMap.iterator());
+    return new IteratorWrapper<>(inMemoryMap.values().iterator(), getDiskBasedMap().iterator());
   }
 
   /**
    * Number of entries in DiskBasedMap
    */
   public int getDiskBasedMapNumEntries() {
-    return diskBasedMap.size();
+    return getDiskBasedMap().size();
   }
 
   /**
    * Number of bytes spilled to disk
    */
   public long getSizeOfFileOnDiskInBytes() {
-    return diskBasedMap.sizeOfFileOnDiskInBytes();
+    return getDiskBasedMap().sizeOfFileOnDiskInBytes();
   }
 
   /**
@@ -112,30 +133,30 @@ public class ExternalSpillableMap<T extends Serializable, R extends Serializable
 
   @Override
   public int size() {
-    return inMemoryMap.size() + diskBasedMap.size();
+    return inMemoryMap.size() + getDiskBasedMap().size();
   }
 
   @Override
   public boolean isEmpty() {
-    return inMemoryMap.isEmpty() && diskBasedMap.isEmpty();
+    return inMemoryMap.isEmpty() && getDiskBasedMap().isEmpty();
   }
 
   @Override
   public boolean containsKey(Object key) {
-    return inMemoryMap.containsKey(key) || diskBasedMap.containsKey(key);
+    return inMemoryMap.containsKey(key) || getDiskBasedMap().containsKey(key);
   }
 
   @Override
   public boolean containsValue(Object value) {
-    return inMemoryMap.containsValue(value) || diskBasedMap.containsValue(value);
+    return inMemoryMap.containsValue(value) || getDiskBasedMap().containsValue(value);
   }
 
   @Override
   public R get(Object key) {
     if (inMemoryMap.containsKey(key)) {
       return inMemoryMap.get(key);
-    } else if (diskBasedMap.containsKey(key)) {
-      return diskBasedMap.get(key);
+    } else if (getDiskBasedMap().containsKey(key)) {
+      return getDiskBasedMap().get(key);
     }
     return null;
   }
@@ -166,19 +187,19 @@ public class ExternalSpillableMap<T extends Serializable, R extends Serializable
       }
       inMemoryMap.put(key, value);
     } else {
-      diskBasedMap.put(key, value);
+      getDiskBasedMap().put(key, value);
     }
     return value;
   }
 
   @Override
   public R remove(Object key) {
-    // NOTE : diskBasedMap.remove does not delete the data from disk
+    // NOTE : getDiskBasedMap().remove does not delete the data from disk
     if (inMemoryMap.containsKey(key)) {
       currentInMemoryMapSize -= estimatedPayloadSize;
       return inMemoryMap.remove(key);
-    } else if (diskBasedMap.containsKey(key)) {
-      return diskBasedMap.remove(key);
+    } else if (getDiskBasedMap().containsKey(key)) {
+      return getDiskBasedMap().remove(key);
     }
     return null;
   }
@@ -193,7 +214,7 @@ public class ExternalSpillableMap<T extends Serializable, R extends Serializable
   @Override
   public void clear() {
     inMemoryMap.clear();
-    diskBasedMap.clear();
+    getDiskBasedMap().clear();
     currentInMemoryMapSize = 0L;
   }
 
@@ -201,23 +222,29 @@ public class ExternalSpillableMap<T extends Serializable, R extends Serializable
   public Set<T> keySet() {
     Set<T> keySet = new HashSet<T>();
     keySet.addAll(inMemoryMap.keySet());
-    keySet.addAll(diskBasedMap.keySet());
+    keySet.addAll(getDiskBasedMap().keySet());
     return keySet;
   }
 
   @Override
   public Collection<R> values() {
-    if (diskBasedMap.isEmpty()) {
+    if (getDiskBasedMap().isEmpty()) {
       return inMemoryMap.values();
     }
-    throw new HoodieNotSupportedException("Cannot return all values in memory");
+    List<R> result = new ArrayList<>(inMemoryMap.values());
+    result.addAll(getDiskBasedMap().values());
+    return result;
+  }
+
+  public Stream<R> valueStream() {
+    return Stream.concat(inMemoryMap.values().stream(), getDiskBasedMap().valueStream());
   }
 
   @Override
   public Set<Entry<T, R>> entrySet() {
     Set<Entry<T, R>> entrySet = new HashSet<>();
     entrySet.addAll(inMemoryMap.entrySet());
-    entrySet.addAll(diskBasedMap.entrySet());
+    entrySet.addAll(getDiskBasedMap().entrySet());
     return entrySet;
   }
 
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/LazyFileIterable.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/LazyFileIterable.java
index 08aa784..75b25be 100644
--- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/LazyFileIterable.java
+++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/collection/LazyFileIterable.java
@@ -16,10 +16,7 @@
 
 package com.uber.hoodie.common.util.collection;
 
-import com.uber.hoodie.common.util.SerializationUtils;
-import com.uber.hoodie.common.util.SpillableMapUtils;
 import com.uber.hoodie.exception.HoodieException;
-import com.uber.hoodie.exception.HoodieIOException;
 import java.io.IOException;
 import java.io.RandomAccessFile;
 import java.util.Iterator;
@@ -34,19 +31,19 @@ import java.util.stream.Collectors;
 public class LazyFileIterable<T, R> implements Iterable<R> {
 
   // Used to access the value written at a specific position in the file
-  private final RandomAccessFile readOnlyFileHandle;
+  private final String filePath;
   // Stores the key and corresponding value's latest metadata spilled to disk
   private final Map<T, DiskBasedMap.ValueMetadata> inMemoryMetadataOfSpilledData;
 
-  public LazyFileIterable(RandomAccessFile file, Map<T, DiskBasedMap.ValueMetadata> map) {
-    this.readOnlyFileHandle = file;
+  public LazyFileIterable(String filePath, Map<T, DiskBasedMap.ValueMetadata> map) {
+    this.filePath = filePath;
     this.inMemoryMetadataOfSpilledData = map;
   }
 
   @Override
   public Iterator<R> iterator() {
     try {
-      return new LazyFileIterator<>(readOnlyFileHandle, inMemoryMetadataOfSpilledData);
+      return new LazyFileIterator<>(filePath, inMemoryMetadataOfSpilledData);
     } catch (IOException io) {
       throw new HoodieException("Unable to initialize iterator for file on disk", io);
     }
@@ -57,11 +54,15 @@ public class LazyFileIterable<T, R> implements Iterable<R> {
    */
   public class LazyFileIterator<T, R> implements Iterator<R> {
 
+    private final String filePath;
     private RandomAccessFile readOnlyFileHandle;
-    private Iterator<Map.Entry<T, DiskBasedMap.ValueMetadata>> metadataIterator;
+    private final Iterator<Map.Entry<T, DiskBasedMap.ValueMetadata>> metadataIterator;
+
+    public LazyFileIterator(String filePath, Map<T, DiskBasedMap.ValueMetadata> map) throws IOException {
+      this.filePath = filePath;
+      this.readOnlyFileHandle = new RandomAccessFile(filePath, "r");
+      readOnlyFileHandle.seek(0);
 
-    public LazyFileIterator(RandomAccessFile file, Map<T, DiskBasedMap.ValueMetadata> map) throws IOException {
-      this.readOnlyFileHandle = file;
       // sort the map in increasing order of offset of value so disk seek is only in one(forward) direction
       this.metadataIterator = map
           .entrySet()
@@ -70,23 +71,25 @@ public class LazyFileIterable<T, R> implements Iterable<R> {
               (Map.Entry<T, DiskBasedMap.ValueMetadata> o1, Map.Entry<T, DiskBasedMap.ValueMetadata> o2) ->
                   o1.getValue().getOffsetOfValue().compareTo(o2.getValue().getOffsetOfValue()))
           .collect(Collectors.toList()).iterator();
+      this.addShutdownHook();
     }
 
     @Override
     public boolean hasNext() {
-      return this.metadataIterator.hasNext();
+      boolean available = this.metadataIterator.hasNext();
+      if (!available) {
+        close();
+      }
+      return available;
     }
 
     @Override
     public R next() {
-      Map.Entry<T, DiskBasedMap.ValueMetadata> entry = this.metadataIterator.next();
-      try {
-        return SerializationUtils.<R>deserialize(SpillableMapUtils
-            .readBytesFromDisk(readOnlyFileHandle, entry.getValue().getOffsetOfValue(),
-                entry.getValue().getSizeOfValue()));
-      } catch (IOException e) {
-        throw new HoodieIOException("Unable to read hoodie record from value spilled to disk", e);
+      if (!hasNext()) {
+        throw new IllegalStateException("next() called on EOF'ed stream. File :" + filePath);
       }
+      Map.Entry<T, DiskBasedMap.ValueMetadata> entry = this.metadataIterator.next();
+      return DiskBasedMap.get(entry.getValue(), readOnlyFileHandle);
     }
 
     @Override
@@ -98,5 +101,24 @@ public class LazyFileIterable<T, R> implements Iterable<R> {
     public void forEachRemaining(Consumer<? super R> action) {
       action.accept(next());
     }
+
+    private void close() {
+      if (readOnlyFileHandle != null) {
+        try {
+          readOnlyFileHandle.close();
+          readOnlyFileHandle = null;
+        } catch (IOException e) {
+          throw new RuntimeException(e);
+        }
+      }
+    }
+
+    private void addShutdownHook() {
+      Runtime.getRuntime().addShutdownHook(new Thread() {
+        public void run() {
+          close();
+        }
+      });
+    }
   }
-}
\ No newline at end of file
+}
diff --git a/hoodie-client/src/main/java/com/uber/hoodie/config/DefaultHoodieConfig.java b/hoodie-common/src/main/java/com/uber/hoodie/config/DefaultHoodieConfig.java
similarity index 100%
rename from hoodie-client/src/main/java/com/uber/hoodie/config/DefaultHoodieConfig.java
rename to hoodie-common/src/main/java/com/uber/hoodie/config/DefaultHoodieConfig.java
diff --git a/hoodie-common/src/main/java/com/uber/hoodie/exception/HoodieRemoteException.java b/hoodie-common/src/main/java/com/uber/hoodie/exception/HoodieRemoteException.java
new file mode 100644
index 0000000..a2f90bb
--- /dev/null
+++ b/hoodie-common/src/main/java/com/uber/hoodie/exception/HoodieRemoteException.java
@@ -0,0 +1,27 @@
+/*
+ * Copyright (c) 2018 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *          http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package com.uber.hoodie.exception;
+
+import java.io.IOException;
+
+public class HoodieRemoteException extends RuntimeException {
+
+  public HoodieRemoteException(IOException t) {
+    super(t.getMessage(), t);
+  }
+
+}
diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java
index 9049d81..e8b287a 100644
--- a/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java
+++ b/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java
@@ -89,9 +89,14 @@ public class HoodieTestUtils {
 
   public static HoodieTableMetaClient init(String basePath)
       throws IOException {
-    return initTableType(getDefaultHadoopConf(), basePath, HoodieTableType.COPY_ON_WRITE);
+    return init(basePath, HoodieTableType.COPY_ON_WRITE);
   }
-  
+
+  public static HoodieTableMetaClient init(String basePath, HoodieTableType tableType)
+      throws IOException {
+    return initTableType(getDefaultHadoopConf(), basePath, tableType);
+  }
+
   public static HoodieTableMetaClient init(Configuration hadoopConf, String basePath)
       throws IOException {
     return initTableType(hadoopConf, basePath, HoodieTableType.COPY_ON_WRITE);
diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/string/MockHoodieTimeline.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/string/MockHoodieTimeline.java
index 83360e4..896f9a5 100644
--- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/string/MockHoodieTimeline.java
+++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/string/MockHoodieTimeline.java
@@ -29,13 +29,13 @@ public class MockHoodieTimeline extends HoodieActiveTimeline {
 
   public MockHoodieTimeline(Stream<String> completed, Stream<String> inflights) throws IOException {
     super();
-    this.instants = Stream.concat(completed.map(s -> new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, s)),
+    this.setInstants(Stream.concat(completed.map(s -> new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, s)),
         inflights.map(s -> new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, s)))
         .sorted(Comparator.comparing(new Function<HoodieInstant, String>() {
           @Override
           public String apply(HoodieInstant hoodieInstant) {
             return hoodieInstant.getFileName();
           }
-        })).collect(Collectors.toList());
+        })).collect(Collectors.toList()));
   }
 }
diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java
index bfd9310..f11acac 100644
--- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java
+++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java
@@ -23,14 +23,17 @@ import static org.junit.Assert.assertTrue;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 import com.uber.hoodie.avro.model.HoodieCompactionPlan;
+import com.uber.hoodie.common.model.CompactionOperation;
 import com.uber.hoodie.common.model.FileSlice;
 import com.uber.hoodie.common.model.HoodieDataFile;
 import com.uber.hoodie.common.model.HoodieFileGroup;
 import com.uber.hoodie.common.model.HoodieFileGroupId;
 import com.uber.hoodie.common.model.HoodieLogFile;
+import com.uber.hoodie.common.model.HoodieTableType;
 import com.uber.hoodie.common.model.HoodieTestUtils;
 import com.uber.hoodie.common.table.HoodieTableMetaClient;
 import com.uber.hoodie.common.table.HoodieTimeline;
+import com.uber.hoodie.common.table.SyncableFileSystemView;
 import com.uber.hoodie.common.table.TableFileSystemView;
 import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
 import com.uber.hoodie.common.table.timeline.HoodieInstant;
@@ -43,6 +46,7 @@ import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
@@ -61,35 +65,38 @@ import org.junit.rules.TemporaryFolder;
 @SuppressWarnings("ResultOfMethodCallIgnored")
 public class HoodieTableFileSystemViewTest {
 
-  private HoodieTableMetaClient metaClient;
-  private String basePath;
-  private HoodieTableFileSystemView fsView;
-  private TableFileSystemView.ReadOptimizedView roView;
-  private TableFileSystemView.RealtimeView rtView;
-  
+  protected HoodieTableMetaClient metaClient;
+  protected String basePath;
+  protected SyncableFileSystemView fsView;
+  protected TableFileSystemView.ReadOptimizedView roView;
+  protected TableFileSystemView.RealtimeView rtView;
+
   @Rule
   public TemporaryFolder tmpFolder = new TemporaryFolder();
 
   @Before
   public void init() throws IOException {
-    metaClient = HoodieTestUtils.init(tmpFolder.getRoot().getAbsolutePath());;
+    initializeMetaClient();
+    refreshFsView();
+  }
+
+  protected void initializeMetaClient() throws IOException {
+    metaClient = HoodieTestUtils.init(tmpFolder.getRoot().getAbsolutePath(), HoodieTableType.MERGE_ON_READ);
     basePath = metaClient.getBasePath();
-    fsView = new HoodieTableFileSystemView(metaClient,
-        metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants());
-    roView = (TableFileSystemView.ReadOptimizedView) fsView;
-    rtView = (TableFileSystemView.RealtimeView) fsView;
   }
 
-  private void refreshFsView(FileStatus[] statuses) {
+  protected SyncableFileSystemView getFileSystemView(HoodieTimeline timeline) throws IOException {
+    return new HoodieTableFileSystemView(metaClient, timeline);
+  }
+
+  protected void refreshFsView() throws IOException {
     metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true);
-    if (statuses != null) {
-      fsView = new HoodieTableFileSystemView(metaClient,
-          metaClient.getActiveTimeline().getCommitsAndCompactionTimeline().filterCompletedAndCompactionInstants(),
-          statuses);
-    } else {
-      fsView = new HoodieTableFileSystemView(metaClient,
-          metaClient.getActiveTimeline().getCommitsAndCompactionTimeline().filterCompletedAndCompactionInstants());
+    if (null != fsView) {
+      fsView.close();
+      fsView = null;
     }
+    fsView = getFileSystemView(
+        metaClient.getActiveTimeline().getCommitsAndCompactionTimeline().filterCompletedAndCompactionInstants());
     roView = (TableFileSystemView.ReadOptimizedView) fsView;
     rtView = (TableFileSystemView.RealtimeView) fsView;
   }
@@ -100,6 +107,11 @@ public class HoodieTableFileSystemViewTest {
    */
   @Test
   public void testViewForFileSlicesWithNoBaseFile() throws Exception {
+    testViewForFileSlicesWithNoBaseFile(1, 0);
+  }
+
+  protected void testViewForFileSlicesWithNoBaseFile(int expNumTotalFileSlices, int expNumTotalDataFiles)
+      throws Exception {
     String partitionPath = "2016/05/01";
     new File(basePath + "/" + partitionPath).mkdirs();
     String fileId = UUID.randomUUID().toString();
@@ -120,7 +132,7 @@ public class HoodieTableFileSystemViewTest {
     commitTimeline.saveAsComplete(deltaInstant2, Optional.empty());
     commitTimeline.saveAsComplete(deltaInstant3, Optional.empty());
 
-    refreshFsView(null);
+    refreshFsView();
 
     List<HoodieDataFile> dataFiles = roView.getLatestDataFiles().collect(Collectors.toList());
     assertTrue("No data file expected", dataFiles.isEmpty());
@@ -159,26 +171,33 @@ public class HoodieTableFileSystemViewTest {
     assertEquals("Correct number of log-files shows up in file-slice", 2, logFiles.size());
     assertEquals("Log File Order check", fileName2, logFiles.get(0).getFileName());
     assertEquals("Log File Order check", fileName1, logFiles.get(1).getFileName());
+
+    assertEquals("Total number of file-slices in view matches expected", expNumTotalFileSlices,
+        rtView.getAllFileSlices(partitionPath).count());
+    assertEquals("Total number of data-files in view matches expected", expNumTotalDataFiles,
+        roView.getAllDataFiles(partitionPath).count());
+    assertEquals("Total number of file-groups in view matches expected", 1,
+        fsView.getAllFileGroups(partitionPath).count());
   }
 
   @Test
   public void testViewForFileSlicesWithNoBaseFileAndRequestedCompaction() throws Exception {
-    testViewForFileSlicesWithAsyncCompaction(true, false);
+    testViewForFileSlicesWithAsyncCompaction(true, false, 2, 1, true);
   }
 
   @Test
   public void testViewForFileSlicesWithBaseFileAndRequestedCompaction() throws Exception {
-    testViewForFileSlicesWithAsyncCompaction(false, false);
+    testViewForFileSlicesWithAsyncCompaction(false, false, 2, 2, true);
   }
 
   @Test
   public void testViewForFileSlicesWithNoBaseFileAndInflightCompaction() throws Exception {
-    testViewForFileSlicesWithAsyncCompaction(true, true);
+    testViewForFileSlicesWithAsyncCompaction(true, true, 2, 1, true);
   }
 
   @Test
   public void testViewForFileSlicesWithBaseFileAndInflightCompaction() throws Exception {
-    testViewForFileSlicesWithAsyncCompaction(false, true);
+    testViewForFileSlicesWithAsyncCompaction(false, true, 2, 2, true);
   }
 
   /**
@@ -210,10 +229,14 @@ public class HoodieTableFileSystemViewTest {
    *                             simulate inserts going directly to log files
    * @param isCompactionInFlight if set, compaction was inflight (running) when view was tested first time,
    *                             otherwise compaction was in requested state
+   * @param  expTotalFileSlices Total number of file-slices across file-groups in the partition path
+   * @param  expTotalDataFiles  Total number of data-files across file-groups in the partition path
+   * @param includeInvalidAndInflight Whether view includes inflight and invalid file-groups.
    * @throws Exception
    */
-  private void testViewForFileSlicesWithAsyncCompaction(boolean skipCreatingDataFile,
-      boolean isCompactionInFlight) throws Exception {
+  protected void testViewForFileSlicesWithAsyncCompaction(boolean skipCreatingDataFile,
+      boolean isCompactionInFlight, int expTotalFileSlices, int expTotalDataFiles,
+      boolean includeInvalidAndInflight) throws Exception {
     String partitionPath = "2016/05/01";
     new File(basePath + "/" + partitionPath).mkdirs();
     String fileId = UUID.randomUUID().toString();
@@ -241,7 +264,7 @@ public class HoodieTableFileSystemViewTest {
     commitTimeline.saveAsComplete(deltaInstant2, Optional.empty());
     commitTimeline.saveAsComplete(deltaInstant3, Optional.empty());
 
-    refreshFsView(null);
+    refreshFsView();
     List<FileSlice> fileSlices = rtView.getLatestFileSlices(partitionPath).collect(Collectors.toList());
     String compactionRequestedTime = "4";
     String compactDataFileName = FSUtils.makeDataFileName(compactionRequestedTime, 1, fileId);
@@ -262,6 +285,15 @@ public class HoodieTableFileSystemViewTest {
       commitTimeline.saveToCompactionRequested(compactionInstant, AvroUtils.serializeCompactionPlan(compactionPlan));
     }
 
+    // View immediately after scheduling compaction
+    refreshFsView();
+    List<FileSlice> slices = rtView.getLatestFileSlices(partitionPath).collect(Collectors.toList());
+    assertEquals("Expected latest file-slices", 1, slices.size());
+    assertEquals("Base-Instant must be compaction Instant", compactionRequestedTime,
+        slices.get(0).getBaseInstantTime());
+    assertFalse("Latest File Slice must not have data-file", slices.get(0).getDataFile().isPresent());
+    assertTrue("Latest File Slice must not have any log-files", slices.get(0).getLogFiles().count() == 0);
+
     // Fake delta-ingestion after compaction-requested
     String deltaInstantTime4 = "5";
     String deltaInstantTime5 = "6";
@@ -275,7 +307,7 @@ public class HoodieTableFileSystemViewTest {
     HoodieInstant deltaInstant5 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime5);
     commitTimeline.saveAsComplete(deltaInstant4, Optional.empty());
     commitTimeline.saveAsComplete(deltaInstant5, Optional.empty());
-    refreshFsView(null);
+    refreshFsView();
 
     List<HoodieDataFile> dataFiles = roView.getAllDataFiles(partitionPath).collect(Collectors.toList());
     if (skipCreatingDataFile) {
@@ -381,7 +413,7 @@ public class HoodieTableFileSystemViewTest {
     // Mark instant as inflight
     commitTimeline.saveToInflight(new HoodieInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION,
         inflightDeltaInstantTime), Optional.empty());
-    refreshFsView(null);
+    refreshFsView();
 
     List<FileSlice> allRawFileSlices = getAllRawFileSlices(partitionPath).collect(Collectors.toList());
     dataFiles = allRawFileSlices.stream().flatMap(slice -> {
@@ -390,58 +422,60 @@ public class HoodieTableFileSystemViewTest {
       }
       return Stream.empty();
     }).collect(Collectors.toList());
-    assertEquals("Inflight/Orphan data-file is also expected", 2
-        + (isCompactionInFlight ? 1 : 0) + (skipCreatingDataFile ? 0 : 1), dataFiles.size());
-    Set<String> fileNames = dataFiles.stream().map(HoodieDataFile::getFileName).collect(Collectors.toSet());
-    assertTrue("Expect orphan data-file to be present", fileNames.contains(orphanDataFileName));
-    assertTrue("Expect inflight data-file to be present", fileNames.contains(inflightDataFileName));
-    if (!skipCreatingDataFile) {
-      assertTrue("Expect old committed data-file", fileNames.contains(dataFileName));
-    }
 
-    if (isCompactionInFlight) {
-      assertTrue("Expect inflight compacted data file to be present", fileNames.contains(compactDataFileName));
-    }
+    if (includeInvalidAndInflight) {
+      assertEquals("Inflight/Orphan data-file is also expected", 2
+          + (isCompactionInFlight ? 1 : 0) + (skipCreatingDataFile ? 0 : 1), dataFiles.size());
+      Set<String> fileNames = dataFiles.stream().map(HoodieDataFile::getFileName).collect(Collectors.toSet());
+      assertTrue("Expect orphan data-file to be present", fileNames.contains(orphanDataFileName));
+      assertTrue("Expect inflight data-file to be present", fileNames.contains(inflightDataFileName));
+      if (!skipCreatingDataFile) {
+        assertTrue("Expect old committed data-file", fileNames.contains(dataFileName));
+      }
 
-    fileSliceList = getLatestRawFileSlices(partitionPath).collect(Collectors.toList());
-    assertEquals("Expect both inflight and orphan file-slice to be included",
-        5, fileSliceList.size());
-    Map<String, FileSlice> fileSliceMap =
-        fileSliceList.stream().collect(Collectors.toMap(FileSlice::getFileId, r -> r));
-    FileSlice orphanFileSliceWithDataFile =  fileSliceMap.get(orphanFileId1);
-    FileSlice orphanFileSliceWithLogFile =  fileSliceMap.get(orphanFileId2);
-    FileSlice inflightFileSliceWithDataFile = fileSliceMap.get(inflightFileId1);
-    FileSlice inflightFileSliceWithLogFile = fileSliceMap.get(inflightFileId2);
-
-    assertEquals("Orphan File Slice with data-file check base-commit", invalidInstantId,
-        orphanFileSliceWithDataFile.getBaseInstantTime());
-    assertEquals("Orphan File Slice with data-file check data-file", orphanDataFileName,
-        orphanFileSliceWithDataFile.getDataFile().get().getFileName());
-    assertEquals("Orphan File Slice with data-file check data-file", 0,
-        orphanFileSliceWithDataFile.getLogFiles().count());
-    assertEquals("Inflight File Slice with data-file check base-commit", inflightDeltaInstantTime,
-        inflightFileSliceWithDataFile.getBaseInstantTime());
-    assertEquals("Inflight File Slice with data-file check data-file", inflightDataFileName,
-        inflightFileSliceWithDataFile.getDataFile().get().getFileName());
-    assertEquals("Inflight File Slice with data-file check data-file", 0,
-        inflightFileSliceWithDataFile.getLogFiles().count());
-    assertEquals("Orphan File Slice with log-file check base-commit", invalidInstantId,
-        orphanFileSliceWithLogFile.getBaseInstantTime());
-    assertFalse("Orphan File Slice with log-file check data-file",
-        orphanFileSliceWithLogFile.getDataFile().isPresent());
-    logFiles = orphanFileSliceWithLogFile.getLogFiles().collect(Collectors.toList());
-    assertEquals("Orphan File Slice with log-file check data-file", 1, logFiles.size());
-    assertEquals("Orphan File Slice with log-file check data-file", orphanLogFileName,
-        logFiles.get(0).getFileName());
-    assertEquals("Inflight File Slice with log-file check base-commit", inflightDeltaInstantTime,
-        inflightFileSliceWithLogFile.getBaseInstantTime());
-    assertFalse("Inflight File Slice with log-file check data-file",
-        inflightFileSliceWithLogFile.getDataFile().isPresent());
-    logFiles = inflightFileSliceWithLogFile.getLogFiles().collect(Collectors.toList());
-    assertEquals("Inflight File Slice with log-file check data-file", 1, logFiles.size());
-    assertEquals("Inflight File Slice with log-file check data-file", inflightLogFileName,
-        logFiles.get(0).getFileName());
+      if (isCompactionInFlight) {
+        assertTrue("Expect inflight compacted data file to be present", fileNames.contains(compactDataFileName));
+      }
 
+      fileSliceList = getLatestRawFileSlices(partitionPath).collect(Collectors.toList());
+      assertEquals("Expect both inflight and orphan file-slice to be included",
+          includeInvalidAndInflight ? 5 : 1, fileSliceList.size());
+      Map<String, FileSlice> fileSliceMap =
+          fileSliceList.stream().collect(Collectors.toMap(FileSlice::getFileId, r -> r));
+      FileSlice orphanFileSliceWithDataFile = fileSliceMap.get(orphanFileId1);
+      FileSlice orphanFileSliceWithLogFile = fileSliceMap.get(orphanFileId2);
+      FileSlice inflightFileSliceWithDataFile = fileSliceMap.get(inflightFileId1);
+      FileSlice inflightFileSliceWithLogFile = fileSliceMap.get(inflightFileId2);
+
+      assertEquals("Orphan File Slice with data-file check base-commit", invalidInstantId,
+          orphanFileSliceWithDataFile.getBaseInstantTime());
+      assertEquals("Orphan File Slice with data-file check data-file", orphanDataFileName,
+          orphanFileSliceWithDataFile.getDataFile().get().getFileName());
+      assertEquals("Orphan File Slice with data-file check data-file", 0,
+          orphanFileSliceWithDataFile.getLogFiles().count());
+      assertEquals("Inflight File Slice with data-file check base-commit", inflightDeltaInstantTime,
+          inflightFileSliceWithDataFile.getBaseInstantTime());
+      assertEquals("Inflight File Slice with data-file check data-file", inflightDataFileName,
+          inflightFileSliceWithDataFile.getDataFile().get().getFileName());
+      assertEquals("Inflight File Slice with data-file check data-file", 0,
+          inflightFileSliceWithDataFile.getLogFiles().count());
+      assertEquals("Orphan File Slice with log-file check base-commit", invalidInstantId,
+          orphanFileSliceWithLogFile.getBaseInstantTime());
+      assertFalse("Orphan File Slice with log-file check data-file",
+          orphanFileSliceWithLogFile.getDataFile().isPresent());
+      logFiles = orphanFileSliceWithLogFile.getLogFiles().collect(Collectors.toList());
+      assertEquals("Orphan File Slice with log-file check data-file", 1, logFiles.size());
+      assertEquals("Orphan File Slice with log-file check data-file", orphanLogFileName,
+          logFiles.get(0).getFileName());
+      assertEquals("Inflight File Slice with log-file check base-commit", inflightDeltaInstantTime,
+          inflightFileSliceWithLogFile.getBaseInstantTime());
+      assertFalse("Inflight File Slice with log-file check data-file",
+          inflightFileSliceWithLogFile.getDataFile().isPresent());
+      logFiles = inflightFileSliceWithLogFile.getLogFiles().collect(Collectors.toList());
+      assertEquals("Inflight File Slice with log-file check data-file", 1, logFiles.size());
+      assertEquals("Inflight File Slice with log-file check data-file", inflightLogFileName,
+          logFiles.get(0).getFileName());
+    }
     // Now simulate Compaction completing - Check the view
     if (!isCompactionInFlight) {
       // For inflight compaction, we already create a data-file to test concurrent inflight case.
@@ -455,14 +489,14 @@ public class HoodieTableFileSystemViewTest {
     }
     compactionInstant = new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionRequestedTime);
     commitTimeline.saveAsComplete(compactionInstant, Optional.empty());
-    refreshFsView(null);
+    refreshFsView();
     // populate the cache
     roView.getAllDataFiles(partitionPath);
 
     fileSliceList = rtView.getLatestFileSlices(partitionPath).collect(Collectors.toList());
+    System.out.println("FILESLICE LIST=" + fileSliceList);
     dataFiles = fileSliceList.stream().map(FileSlice::getDataFile)
         .filter(Optional::isPresent).map(Optional::get).collect(Collectors.toList());
-    System.out.println("fileSliceList : " + fileSliceList);
     assertEquals("Expect only one data-files in latest view as there is only one file-group", 1, dataFiles.size());
     assertEquals("Data Filename must match", compactDataFileName, dataFiles.get(0).getFileName());
     assertEquals("Only one latest file-slice in the partition", 1, fileSliceList.size());
@@ -502,6 +536,14 @@ public class HoodieTableFileSystemViewTest {
       assertEquals("Expect data-file created by compaction be returned", df.getCommitTime(),
           compactionRequestedTime);
     });
+
+    assertEquals("Total number of file-slices in partitions matches expected", expTotalFileSlices,
+        rtView.getAllFileSlices(partitionPath).count());
+    assertEquals("Total number of data-files in partitions matches expected", expTotalDataFiles,
+        roView.getAllDataFiles(partitionPath).count());
+    // file-groups includes inflight/invalid file-ids
+    assertEquals("Total number of file-groups in partitions matches expected",
+        5, fsView.getAllFileGroups(partitionPath).count());
   }
 
   @Test
@@ -518,7 +560,7 @@ public class HoodieTableFileSystemViewTest {
     String commitTime1 = "1";
     String fileName1 = FSUtils.makeDataFileName(commitTime1, 1, fileId);
     new File(basePath + "/" + partitionPath + "/" + fileName1).createNewFile();
-    refreshFsView(null);
+    refreshFsView();
     assertFalse("No commit, should not find any data file",
         roView.getLatestDataFiles(partitionPath).filter(dfile -> dfile.getFileId().equals(fileId)).findFirst()
             .isPresent());
@@ -527,7 +569,7 @@ public class HoodieTableFileSystemViewTest {
     HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline();
     HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime1);
     commitTimeline.saveAsComplete(instant1, Optional.empty());
-    refreshFsView(null);
+    refreshFsView();
     assertEquals("", fileName1,
         roView.getLatestDataFiles(partitionPath).filter(dfile -> dfile.getFileId().equals(fileId)).findFirst().get()
             .getFileName());
@@ -536,7 +578,7 @@ public class HoodieTableFileSystemViewTest {
     String commitTime2 = "2";
     String fileName2 = FSUtils.makeDataFileName(commitTime2, 1, fileId);
     new File(basePath + "/" + partitionPath + "/" + fileName2).createNewFile();
-    refreshFsView(null);
+    refreshFsView();
     assertEquals("", fileName1,
         roView.getLatestDataFiles(partitionPath).filter(dfile -> dfile.getFileId().equals(fileId)).findFirst().get()
             .getFileName());
@@ -544,7 +586,7 @@ public class HoodieTableFileSystemViewTest {
     // Make it safe
     HoodieInstant instant2 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime2);
     commitTimeline.saveAsComplete(instant2, Optional.empty());
-    refreshFsView(null);
+    refreshFsView();
     assertEquals("", fileName2,
         roView.getLatestDataFiles(partitionPath).filter(dfile -> dfile.getFileId().equals(fileId)).findFirst().get()
             .getFileName());
@@ -552,6 +594,10 @@ public class HoodieTableFileSystemViewTest {
 
   @Test
   public void testStreamLatestVersionInPartition() throws IOException {
+    testStreamLatestVersionInPartition(false);
+  }
+
+  public void testStreamLatestVersionInPartition(boolean isLatestFileSliceOnly) throws IOException {
     // Put some files in the partition
     String fullPartitionPath = basePath + "/2016/05/01/";
     new File(fullPartitionPath).mkdirs();
@@ -588,16 +634,16 @@ public class HoodieTableFileSystemViewTest {
     // Now we list the entire partition
     FileStatus[] statuses = metaClient.getFs().listStatus(new Path(fullPartitionPath));
     assertEquals(11, statuses.length);
-    refreshFsView(null);
+    refreshFsView();
 
     // Check files as of lastest commit.
     List<FileSlice> allSlices = rtView.getAllFileSlices("2016/05/01").collect(Collectors.toList());
-    assertEquals(8, allSlices.size());
+    assertEquals(isLatestFileSliceOnly ? 4 : 8, allSlices.size());
     Map<String, Long> fileSliceMap = allSlices.stream().collect(
         Collectors.groupingBy(slice -> slice.getFileId(), Collectors.counting()));
-    assertEquals(2, fileSliceMap.get(fileId1).longValue());
-    assertEquals(3, fileSliceMap.get(fileId2).longValue());
-    assertEquals(2, fileSliceMap.get(fileId3).longValue());
+    assertEquals(isLatestFileSliceOnly ? 1 : 2, fileSliceMap.get(fileId1).longValue());
+    assertEquals(isLatestFileSliceOnly ? 1 : 3, fileSliceMap.get(fileId2).longValue());
+    assertEquals(isLatestFileSliceOnly ? 1 : 2, fileSliceMap.get(fileId3).longValue());
     assertEquals(1, fileSliceMap.get(fileId4).longValue());
 
     List<HoodieDataFile> dataFileList = roView.getLatestDataFilesBeforeOrOn("2016/05/01", commitTime4)
@@ -627,24 +673,35 @@ public class HoodieTableFileSystemViewTest {
     // Reset the max commit time
     List<HoodieDataFile> dataFiles = roView.getLatestDataFilesBeforeOrOn("2016/05/01", commitTime3)
         .collect(Collectors.toList());
-    assertEquals(dataFiles.size(), 3);
     filenames = Sets.newHashSet();
     for (HoodieDataFile status : dataFiles) {
       filenames.add(status.getFileName());
     }
-    assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime1, 1, fileId1)));
-    assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId2)));
-    assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId3)));
+    if (!isLatestFileSliceOnly) {
+      assertEquals(3, dataFiles.size());
+      assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime1, 1, fileId1)));
+      assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId2)));
+      assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId3)));
+    } else {
+      assertEquals(1, dataFiles.size());
+      assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId2)));
+    }
 
-    logFilesList = rtView.getLatestFileSlicesBeforeOrOn("2016/05/01", commitTime3).map(slice -> slice.getLogFiles())
-        .flatMap(logFileList -> logFileList).collect(Collectors.toList());
+    logFilesList =
+        rtView.getLatestFileSlicesBeforeOrOn("2016/05/01", commitTime3).map(slice -> slice.getLogFiles())
+            .flatMap(logFileList -> logFileList).collect(Collectors.toList());
     assertEquals(logFilesList.size(), 1);
     assertTrue(logFilesList.get(0).getFileName()
         .equals(FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime3, 0)));
+
   }
 
   @Test
   public void testStreamEveryVersionInPartition() throws IOException {
+    testStreamEveryVersionInPartition(false);
+  }
+
+  protected void testStreamEveryVersionInPartition(boolean isLatestFileSliceOnly) throws IOException {
     // Put some files in the partition
     String fullPartitionPath = basePath + "/2016/05/01/";
     new File(fullPartitionPath).mkdirs();
@@ -673,7 +730,7 @@ public class HoodieTableFileSystemViewTest {
     FileStatus[] statuses = metaClient.getFs().listStatus(new Path(fullPartitionPath));
     assertEquals(7, statuses.length);
 
-    refreshFsView(null);
+    refreshFsView();
     List<HoodieFileGroup> fileGroups = fsView.getAllFileGroups("2016/05/01").collect(Collectors.toList());
     assertEquals(3, fileGroups.size());
 
@@ -684,21 +741,36 @@ public class HoodieTableFileSystemViewTest {
         assertEquals("All same fileId should be grouped", fileId, dataFile.getFileId());
         filenames.add(dataFile.getFileName());
       });
+      Set<String> expFileNames = new HashSet<>();
       if (fileId.equals(fileId1)) {
-        assertEquals(filenames, Sets.newHashSet(FSUtils.makeDataFileName(commitTime1, 1, fileId1),
-            FSUtils.makeDataFileName(commitTime4, 1, fileId1)));
+        if (!isLatestFileSliceOnly) {
+          expFileNames.add(FSUtils.makeDataFileName(commitTime1, 1, fileId1));
+        }
+        expFileNames.add(FSUtils.makeDataFileName(commitTime4, 1, fileId1));
+        assertEquals(expFileNames, filenames);
       } else if (fileId.equals(fileId2)) {
-        assertEquals(filenames, Sets.newHashSet(FSUtils.makeDataFileName(commitTime1, 1, fileId2),
-            FSUtils.makeDataFileName(commitTime2, 1, fileId2), FSUtils.makeDataFileName(commitTime3, 1, fileId2)));
+        if (!isLatestFileSliceOnly) {
+          expFileNames.add(FSUtils.makeDataFileName(commitTime1, 1, fileId2));
+          expFileNames.add(FSUtils.makeDataFileName(commitTime2, 1, fileId2));
+        }
+        expFileNames.add(FSUtils.makeDataFileName(commitTime3, 1, fileId2));
+        assertEquals(expFileNames, filenames);
       } else {
-        assertEquals(filenames, Sets.newHashSet(FSUtils.makeDataFileName(commitTime3, 1, fileId3),
-            FSUtils.makeDataFileName(commitTime4, 1, fileId3)));
+        if (!isLatestFileSliceOnly) {
+          expFileNames.add(FSUtils.makeDataFileName(commitTime3, 1, fileId3));
+        }
+        expFileNames.add(FSUtils.makeDataFileName(commitTime4, 1, fileId3));
+        assertEquals(expFileNames, filenames);
       }
     }
   }
 
   @Test
-  public void streamLatestVersionInRange() throws IOException {
+  public void testStreamLatestVersionInRange() throws IOException {
+    testStreamLatestVersionInRange(false);
+  }
+
+  protected void testStreamLatestVersionInRange(boolean isLatestFileSliceOnly) throws IOException {
     // Put some files in the partition
     String fullPartitionPath = basePath + "/2016/05/01/";
     new File(fullPartitionPath).mkdirs();
@@ -718,7 +790,7 @@ public class HoodieTableFileSystemViewTest {
     new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2)).createNewFile();
     new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2)).createNewFile();
     new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2)).createNewFile();
-    new File(fullPartitionPath + FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0))
+    new File(fullPartitionPath + FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime3, 0))
         .createNewFile();
 
     new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3)).createNewFile();
@@ -733,16 +805,23 @@ public class HoodieTableFileSystemViewTest {
     FileStatus[] statuses = metaClient.getFs().listStatus(new Path(fullPartitionPath));
     assertEquals(9, statuses.length);
 
-    refreshFsView(statuses);
+    refreshFsView();
+    // Populate view for partition
+    roView.getAllDataFiles("2016/05/01/");
+
     List<HoodieDataFile> dataFiles = roView.getLatestDataFilesInRange(Lists.newArrayList(commitTime2, commitTime3))
         .collect(Collectors.toList());
-    assertEquals(3, dataFiles.size());
+    assertEquals(isLatestFileSliceOnly ? 2 : 3, dataFiles.size());
     Set<String> filenames = Sets.newHashSet();
     for (HoodieDataFile status : dataFiles) {
       filenames.add(status.getFileName());
     }
+
+    assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId1)));
     assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId2)));
-    assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId3)));
+    if (!isLatestFileSliceOnly) {
+      assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId3)));
+    }
 
     List<FileSlice> slices = rtView.getLatestFileSliceInRange(Lists.newArrayList(commitTime3, commitTime4))
         .collect(Collectors.toList());
@@ -753,8 +832,8 @@ public class HoodieTableFileSystemViewTest {
         assertTrue(slice.getDataFile().isPresent());
         assertEquals(slice.getLogFiles().count(), 0);
       } else if (slice.getFileId().equals(fileId2)) {
-        assertEquals(slice.getBaseInstantTime(), commitTime4);
-        assertFalse(slice.getDataFile().isPresent());
+        assertEquals(slice.getBaseInstantTime(), commitTime3);
+        assertTrue(slice.getDataFile().isPresent());
         assertEquals(slice.getLogFiles().count(), 1);
       } else if (slice.getFileId().equals(fileId3)) {
         assertEquals(slice.getBaseInstantTime(), commitTime4);
@@ -765,7 +844,11 @@ public class HoodieTableFileSystemViewTest {
   }
 
   @Test
-  public void streamLatestVersionsBefore() throws IOException {
+  public void testStreamLatestVersionsBefore() throws IOException {
+    testStreamLatestVersionsBefore(false);
+  }
+
+  protected void testStreamLatestVersionsBefore(boolean isLatestFileSliceOnly) throws IOException {
     // Put some files in the partition
     String partitionPath = "2016/05/01/";
     String fullPartitionPath = basePath + "/" + partitionPath;
@@ -795,22 +878,30 @@ public class HoodieTableFileSystemViewTest {
     FileStatus[] statuses = metaClient.getFs().listStatus(new Path(fullPartitionPath));
     assertEquals(7, statuses.length);
 
-    refreshFsView(null);
+    refreshFsView();
     List<HoodieDataFile> dataFiles = roView.getLatestDataFilesBeforeOrOn(partitionPath, commitTime2)
         .collect(Collectors.toList());
-    assertEquals(2, dataFiles.size());
-    Set<String> filenames = Sets.newHashSet();
-    for (HoodieDataFile status : dataFiles) {
-      filenames.add(status.getFileName());
+    if (!isLatestFileSliceOnly) {
+      assertEquals(2, dataFiles.size());
+      Set<String> filenames = Sets.newHashSet();
+      for (HoodieDataFile status : dataFiles) {
+        filenames.add(status.getFileName());
+      }
+      assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime1, 1, fileId1)));
+      assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime2, 1, fileId2)));
+    } else {
+      assertEquals(0, dataFiles.size());
     }
-    assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime1, 1, fileId1)));
-    assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime2, 1, fileId2)));
   }
 
   @Test
-  public void streamLatestVersions() throws IOException {
+  public void testStreamLatestVersions() throws IOException {
+    testStreamLatestVersions(false);
+  }
+
+  protected void testStreamLatestVersions(boolean isLatestFileSliceOnly) throws IOException {
     // Put some files in the partition
-    String partitionPath = "2016/05/01/";
+    String partitionPath = "2016/05/01";
     String fullPartitionPath = basePath + "/" + partitionPath;
     new File(fullPartitionPath).mkdirs();
     String commitTime1 = "1";
@@ -821,21 +912,28 @@ public class HoodieTableFileSystemViewTest {
     String fileId2 = UUID.randomUUID().toString();
     String fileId3 = UUID.randomUUID().toString();
 
-    new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1)).createNewFile();
-    new File(fullPartitionPath + FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime1, 0))
-        .createNewFile();
-    new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId1)).createNewFile();
-    new File(fullPartitionPath + FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0))
+    new File(fullPartitionPath + "/" + FSUtils.makeDataFileName(commitTime1, 1, fileId1)).createNewFile();
+    new File(fullPartitionPath + "/"
+        + FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime1, 0)).createNewFile();
+    new File(fullPartitionPath + "/"
+        + FSUtils.makeDataFileName(commitTime4, 1, fileId1)).createNewFile();
+    new File(fullPartitionPath + "/"
+        + FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0)).createNewFile();
+
+    new File(fullPartitionPath + "/"
+        + FSUtils.makeDataFileName(commitTime1, 1, fileId2)).createNewFile();
+    new File(fullPartitionPath + "/"
+        + FSUtils.makeDataFileName(commitTime2, 1, fileId2)).createNewFile();
+    new File(fullPartitionPath + "/"
+        + FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime2, 0))
         .createNewFile();
+    new File(fullPartitionPath + "/"
+        + FSUtils.makeDataFileName(commitTime3, 1, fileId2)).createNewFile();
 
-    new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2)).createNewFile();
-    new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2)).createNewFile();
-    new File(fullPartitionPath + FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime2, 0))
-        .createNewFile();
-    new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2)).createNewFile();
-
-    new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3)).createNewFile();
-    new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3)).createNewFile();
+    new File(fullPartitionPath + "/"
+        + FSUtils.makeDataFileName(commitTime3, 1, fileId3)).createNewFile();
+    new File(fullPartitionPath + "/"
+        + FSUtils.makeDataFileName(commitTime4, 1, fileId3)).createNewFile();
 
     new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile();
     new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile();
@@ -846,26 +944,32 @@ public class HoodieTableFileSystemViewTest {
     FileStatus[] statuses = metaClient.getFs().listStatus(new Path(fullPartitionPath));
     assertEquals(10, statuses.length);
 
-    refreshFsView(statuses);
-
+    refreshFsView();
+    fsView.getAllDataFiles(partitionPath);
     List<HoodieFileGroup> fileGroups = fsView.getAllFileGroups(partitionPath).collect(Collectors.toList());
     assertEquals(3, fileGroups.size());
     for (HoodieFileGroup fileGroup : fileGroups) {
       List<FileSlice> slices = fileGroup.getAllFileSlices().collect(Collectors.toList());
       String fileId = fileGroup.getFileGroupId().getFileId();
       if (fileId.equals(fileId1)) {
-        assertEquals(2, slices.size());
+        assertEquals(isLatestFileSliceOnly ? 1 : 2, slices.size());
         assertEquals(commitTime4, slices.get(0).getBaseInstantTime());
-        assertEquals(commitTime1, slices.get(1).getBaseInstantTime());
+        if (!isLatestFileSliceOnly) {
+          assertEquals(commitTime1, slices.get(1).getBaseInstantTime());
+        }
       } else if (fileId.equals(fileId2)) {
-        assertEquals(3, slices.size());
+        assertEquals(isLatestFileSliceOnly ? 1 : 3, slices.size());
         assertEquals(commitTime3, slices.get(0).getBaseInstantTime());
-        assertEquals(commitTime2, slices.get(1).getBaseInstantTime());
-        assertEquals(commitTime1, slices.get(2).getBaseInstantTime());
+        if (!isLatestFileSliceOnly) {
+          assertEquals(commitTime2, slices.get(1).getBaseInstantTime());
+          assertEquals(commitTime1, slices.get(2).getBaseInstantTime());
+        }
       } else if (fileId.equals(fileId3)) {
-        assertEquals(2, slices.size());
+        assertEquals(isLatestFileSliceOnly ? 1 : 2, slices.size());
         assertEquals(commitTime4, slices.get(0).getBaseInstantTime());
-        assertEquals(commitTime3, slices.get(1).getBaseInstantTime());
+        if (!isLatestFileSliceOnly) {
+          assertEquals(commitTime3, slices.get(1).getBaseInstantTime());
+        }
       }
     }
 
@@ -925,9 +1029,11 @@ public class HoodieTableFileSystemViewTest {
         new Path(fullPartitionPath1), new Path(fullPartitionPath2), new Path(fullPartitionPath3)
     });
     assertEquals(6, statuses.length);
-    refreshFsView(statuses);
+    refreshFsView();
+    Arrays.asList(partitionPath1, partitionPath2, partitionPath3).forEach(p -> fsView.getAllFileGroups(p).count());
 
-    List<HoodieFileGroup> groups = fsView.getAllFileGroups().collect(Collectors.toList());
+    List<HoodieFileGroup> groups = Stream.of(partitionPath1, partitionPath2, partitionPath3)
+        .flatMap(p -> fsView.getAllFileGroups(p)).collect(Collectors.toList());
     Assert.assertEquals("Expected number of file-groups", 3, groups.size());
     Assert.assertEquals("Partitions must be different for file-groups", 3,
         groups.stream().map(HoodieFileGroup::getPartitionPath).collect(Collectors.toSet()).size());
@@ -979,7 +1085,7 @@ public class HoodieTableFileSystemViewTest {
     HoodieInstant deltaInstant5 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime5);
     commitTimeline.saveAsComplete(deltaInstant4, Optional.empty());
     commitTimeline.saveAsComplete(deltaInstant5, Optional.empty());
-    refreshFsView(null);
+    refreshFsView();
 
     // Test Data Files
     List<HoodieDataFile> dataFiles = roView.getAllDataFiles(partitionPath1).collect(Collectors.toList());
@@ -1019,18 +1125,16 @@ public class HoodieTableFileSystemViewTest {
       assertEquals("Log File Order check", fileName3, logFiles.get(1).getFileName());
     });
 
-    Assert.assertEquals(3, fsView.getFgIdToPendingCompaction().size());
-    Set<String> partitionsInCompaction =
-        fsView.getFgIdToPendingCompaction().keySet().stream().map(HoodieFileGroupId::getPartitionPath)
-            .collect(Collectors.toSet());
+    Assert.assertEquals(3, fsView.getPendingCompactionOperations().count());
+    Set<String> partitionsInCompaction = fsView.getPendingCompactionOperations().map(Pair::getValue)
+        .map(CompactionOperation::getPartitionPath).collect(Collectors.toSet());
     Assert.assertEquals(3, partitionsInCompaction.size());
     Assert.assertTrue(partitionsInCompaction.contains(partitionPath1));
     Assert.assertTrue(partitionsInCompaction.contains(partitionPath2));
     Assert.assertTrue(partitionsInCompaction.contains(partitionPath3));
 
-    Set<String> fileIdsInCompaction =
-        fsView.getFgIdToPendingCompaction().keySet().stream().map(HoodieFileGroupId::getFileId)
-            .collect(Collectors.toSet());
+    Set<String> fileIdsInCompaction = fsView.getPendingCompactionOperations().map(Pair::getValue)
+            .map(CompactionOperation::getFileId).collect(Collectors.toSet());
     Assert.assertEquals(1, fileIdsInCompaction.size());
     Assert.assertTrue(fileIdsInCompaction.contains(fileId));
   }
diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/IncrementalFSViewSyncTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/IncrementalFSViewSyncTest.java
new file mode 100644
index 0000000..dc31c1a
--- /dev/null
+++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/IncrementalFSViewSyncTest.java
@@ -0,0 +1,783 @@
+/*
+ *  Copyright (c) 2019 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
+ *
+ *  Licensed under the Apache License, Version 2.0 (the "License");
+ *  you may not use this file except in compliance with the License.
+ *  You may obtain a copy of the License at
+ *
+ *           http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package com.uber.hoodie.common.table.view;
+
+import static com.uber.hoodie.common.table.HoodieTimeline.COMPACTION_ACTION;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Iterators;
+import com.uber.hoodie.avro.model.HoodieCleanMetadata;
+import com.uber.hoodie.avro.model.HoodieCompactionPlan;
+import com.uber.hoodie.avro.model.HoodieRestoreMetadata;
+import com.uber.hoodie.avro.model.HoodieRollbackMetadata;
+import com.uber.hoodie.common.HoodieCleanStat;
+import com.uber.hoodie.common.HoodieRollbackStat;
+import com.uber.hoodie.common.model.CompactionOperation;
+import com.uber.hoodie.common.model.FileSlice;
+import com.uber.hoodie.common.model.HoodieCleaningPolicy;
+import com.uber.hoodie.common.model.HoodieCommitMetadata;
+import com.uber.hoodie.common.model.HoodieDataFile;
+import com.uber.hoodie.common.model.HoodieFileGroup;
+import com.uber.hoodie.common.model.HoodieFileGroupId;
+import com.uber.hoodie.common.model.HoodieTableType;
+import com.uber.hoodie.common.model.HoodieTestUtils;
+import com.uber.hoodie.common.model.HoodieWriteStat;
+import com.uber.hoodie.common.table.HoodieTableMetaClient;
+import com.uber.hoodie.common.table.HoodieTimeline;
+import com.uber.hoodie.common.table.SyncableFileSystemView;
+import com.uber.hoodie.common.table.timeline.HoodieInstant;
+import com.uber.hoodie.common.table.timeline.HoodieInstant.State;
+import com.uber.hoodie.common.util.AvroUtils;
+import com.uber.hoodie.common.util.CompactionUtils;
+import com.uber.hoodie.common.util.FSUtils;
+import com.uber.hoodie.common.util.collection.Pair;
+import com.uber.hoodie.exception.HoodieException;
+import java.io.File;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import org.apache.hadoop.fs.Path;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class IncrementalFSViewSyncTest {
+
+  protected HoodieTableMetaClient metaClient;
+  protected String basePath;
+
+  private final List<String> partitions = Arrays.asList("2018/01/01", "2018/01/02",
+      "2019/03/01");
+  private final List<String> fileIdsPerPartition =
+      IntStream.range(0, 10).mapToObj(x -> UUID.randomUUID().toString()).collect(Collectors.toList());
+
+  @Rule
+  public TemporaryFolder tmpFolder = new TemporaryFolder();
+
+  @Before
+  public void init() throws IOException {
+    initializeMetaClient();
+    refreshFsView();
+  }
+
+  protected void initializeMetaClient() throws IOException {
+    metaClient = HoodieTestUtils.init(tmpFolder.getRoot().getAbsolutePath(), HoodieTableType.MERGE_ON_READ);
+    basePath = metaClient.getBasePath();
+    partitions.forEach(p -> new File(basePath + "/" + p).mkdirs());
+  }
+
+  protected void refreshFsView() throws IOException {
+    metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true);
+  }
+
+  protected SyncableFileSystemView getNewFileSystemView(HoodieTableMetaClient metaClient) throws IOException {
+    return getNewFileSystemView(metaClient, metaClient.getActiveTimeline().filterCompletedAndCompactionInstants());
+  }
+
+  protected SyncableFileSystemView getNewFileSystemView(HoodieTableMetaClient metaClient, HoodieTimeline timeline)
+      throws IOException {
+    return new HoodieTableFileSystemView(metaClient, timeline, true);
+  }
+
+  @Test
+  public void testEmptyPartitionsAndTimeline() throws IOException {
+    SyncableFileSystemView view = getNewFileSystemView(metaClient);
+    Assert.assertFalse(view.getLastInstant().isPresent());
+    partitions.forEach(p -> Assert.assertEquals(0, view.getLatestFileSlices(p).count()));
+  }
+
+  @Test
+  public void testAsyncCompaction() throws IOException {
... 2909 lines suppressed ...