You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@samza.apache.org by pm...@apache.org on 2021/05/25 23:00:57 UTC

[samza] branch state-backend-async-commit updated: SAMZA-2657: Blob Store as backend for Samza State backup and restore (#1501)

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

pmaheshwari pushed a commit to branch state-backend-async-commit
in repository https://gitbox.apache.org/repos/asf/samza.git


The following commit(s) were added to refs/heads/state-backend-async-commit by this push:
     new 7cc4eaa  SAMZA-2657: Blob Store as backend for Samza State backup and restore (#1501)
7cc4eaa is described below

commit 7cc4eaa96fff244f6dce9c18af804917db7c3b2b
Author: shekhars-li <72...@users.noreply.github.com>
AuthorDate: Tue May 25 16:00:46 2021 -0700

    SAMZA-2657: Blob Store as backend for Samza State backup and restore (#1501)
    
    SAMZA-2657: Introduce blob store as backend for Samza state backup and restore
---
 build.gradle                                       |   1 +
 ...torageAdmin.java => BlobStoreAdminFactory.java} |  19 +-
 ...askStorageAdmin.java => StateBackendAdmin.java} |  11 +-
 .../apache/samza/storage/StateBackendFactory.java  |   8 +-
 .../samza/storage/blobstore/BlobStoreManager.java  |  83 ++
 .../BlobStoreManagerFactory.java}                  |  15 +-
 .../apache/samza/storage/blobstore/Metadata.java   | 114 +++
 .../exceptions/DeletedException.java}              |  26 +-
 .../exceptions/RetriableException.java}            |  26 +-
 .../clustermanager/ClusterBasedJobCoordinator.java |  15 +
 .../org/apache/samza/config/BlobStoreConfig.java   |  21 +-
 .../org/apache/samza/config/StorageConfig.java     |  74 +-
 .../standalone/PassthroughJobCoordinator.java      |  14 +-
 .../storage/KafkaChangelogStateBackendFactory.java |  70 +-
 .../NonTransactionalStateTaskRestoreManager.java   |   2 +-
 .../apache/samza/storage/StorageManagerUtil.java   |  25 +-
 .../org/apache/samza/storage/StorageRecovery.java  |   2 +-
 .../samza/storage/TaskSideInputStorageManager.java |  17 +-
 .../samza/storage/TaskStorageCommitManager.java    |   7 +-
 .../TransactionalStateTaskRestoreManager.java      |   3 +-
 .../storage/blobstore/BlobStoreBackupManager.java  | 350 ++++++++
 .../storage/blobstore/BlobStoreRestoreManager.java | 369 ++++++++
 .../blobstore/BlobStoreStateBackendFactory.java    |  98 +++
 .../samza/storage/blobstore/diff/DirDiff.java      | 244 ++++++
 .../samza/storage/blobstore/index/DirIndex.java    | 176 ++++
 .../samza/storage/blobstore/index/FileBlob.java    |  85 ++
 .../samza/storage/blobstore/index/FileIndex.java   | 114 +++
 .../storage/blobstore/index/FileMetadata.java      | 138 +++
 .../storage/blobstore/index/SnapshotIndex.java     | 124 +++
 .../storage/blobstore/index/SnapshotMetadata.java  | 115 +++
 .../blobstore/index/serde/JsonDirIndexMixin.java   |  58 ++
 .../blobstore/index/serde/JsonFileBlobMixin.java   |  21 +-
 .../blobstore/index/serde/JsonFileIndexMixin.java  |  53 ++
 .../index/serde/JsonFileMetadataMixin.java         |  55 ++
 .../index/serde/JsonSnapshotIndexMixin.java        |  53 ++
 .../index/serde/JsonSnapshotMetadataMixin.java     |  53 ++
 .../blobstore/index/serde/SnapshotIndexSerde.java  |  76 ++
 .../metrics/BlobStoreBackupManagerMetrics.java     | 136 +++
 .../metrics/BlobStoreRestoreManagerMetrics.java    |  94 +++
 .../storage/blobstore/util/BlobStoreUtil.java      | 622 ++++++++++++++
 .../samza/storage/blobstore/util/DirDiffUtil.java  | 403 +++++++++
 .../java/org/apache/samza/util/FutureUtil.java     |   8 +-
 .../apache/samza/container/SamzaContainer.scala    |  73 +-
 .../apache/samza/job/local/ProcessJobFactory.scala |  15 +-
 .../samza/storage/ContainerStorageManager.java     |   4 +-
 .../org/apache/samza/config/TestStorageConfig.java |  40 +-
 .../storage/TestTaskStorageCommitManager.java      |  68 +-
 .../blobstore/TestBlobStoreBackupManager.java      | 542 ++++++++++++
 .../blobstore/TestBlobStoreRestoreManager.java     | 361 ++++++++
 .../blobstore/serde/TestSnapshotIndexSerde.java    |  56 ++
 .../storage/blobstore/util/BlobStoreTestUtil.java  | 296 +++++++
 .../storage/blobstore/util/TestBlobStoreUtil.java  | 936 +++++++++++++++++++++
 .../storage/blobstore/util/TestDirDiffUtil.java    | 127 +++
 .../blobstore/util/TestDirDiffUtilMisc.java        |  71 ++
 .../samza/storage/kv/RocksDbOptionsHelper.java     | 124 ++-
 .../kv/RocksDbKeyValueStorageEngineFactory.scala   |   8 +-
 .../samza/storage/kv/RocksDbKeyValueStore.scala    |  12 +-
 .../kv/BaseKeyValueStorageEngineFactory.java       |   2 +-
 .../kv/TestBaseKeyValueStorageEngineFactory.java   |   2 +-
 59 files changed, 6507 insertions(+), 228 deletions(-)

diff --git a/build.gradle b/build.gradle
index 9c48b23..a3512e0 100644
--- a/build.gradle
+++ b/build.gradle
@@ -191,6 +191,7 @@ project(":samza-core_$scalaSuffix") {
     compile "org.apache.commons:commons-lang3:$commonsLang3Version"
     compile "commons-io:commons-io:$commonsIoVersion"
     compile "com.fasterxml.jackson.core:jackson-databind:$jacksonVersion"
+    compile "com.fasterxml.jackson.datatype:jackson-datatype-jdk8:$jacksonVersion"
     compile "org.eclipse.jetty:jetty-webapp:$jettyVersion"
     compile "org.scala-lang:scala-library:$scalaVersion"
     compile "org.slf4j:slf4j-api:$slf4jVersion"
diff --git a/samza-api/src/main/java/org/apache/samza/storage/TaskStorageAdmin.java b/samza-api/src/main/java/org/apache/samza/storage/BlobStoreAdminFactory.java
similarity index 65%
copy from samza-api/src/main/java/org/apache/samza/storage/TaskStorageAdmin.java
copy to samza-api/src/main/java/org/apache/samza/storage/BlobStoreAdminFactory.java
index 205077b..6070ef1 100644
--- a/samza-api/src/main/java/org/apache/samza/storage/TaskStorageAdmin.java
+++ b/samza-api/src/main/java/org/apache/samza/storage/BlobStoreAdminFactory.java
@@ -19,12 +19,19 @@
 
 package org.apache.samza.storage;
 
-/**
- * Creates and validate resources for the StateBackendFactory
- */
-public interface TaskStorageAdmin {
+import org.apache.samza.config.Config;
+import org.apache.samza.job.model.JobModel;
 
-  void createResources();
 
-  void validateResources();
+/**
+ * Factory to create instance of {@link StateBackendAdmin}s that needs to be implemented for every
+ * state backend
+ */
+public interface BlobStoreAdminFactory {
+  /**
+   * Returns an instance of {@link StateBackendAdmin}
+   * @param config job configuration
+   * @param jobModel Job Model
+   */
+  StateBackendAdmin getStateBackendAdmin(Config config, JobModel jobModel);
 }
diff --git a/samza-api/src/main/java/org/apache/samza/storage/TaskStorageAdmin.java b/samza-api/src/main/java/org/apache/samza/storage/StateBackendAdmin.java
similarity index 76%
copy from samza-api/src/main/java/org/apache/samza/storage/TaskStorageAdmin.java
copy to samza-api/src/main/java/org/apache/samza/storage/StateBackendAdmin.java
index 205077b..2cdb86f 100644
--- a/samza-api/src/main/java/org/apache/samza/storage/TaskStorageAdmin.java
+++ b/samza-api/src/main/java/org/apache/samza/storage/StateBackendAdmin.java
@@ -20,11 +20,16 @@
 package org.apache.samza.storage;
 
 /**
- * Creates and validate resources for the StateBackendFactory
+ * Admin responsible for loading any resources related to state backend
  */
-public interface TaskStorageAdmin {
-
+public interface StateBackendAdmin {
+  /**
+   * Create all the resources required per job per store state backend
+   */
   void createResources();
 
+  /**
+   * Validate all resources required per job per state for state backend
+   */
   void validateResources();
 }
diff --git a/samza-api/src/main/java/org/apache/samza/storage/StateBackendFactory.java b/samza-api/src/main/java/org/apache/samza/storage/StateBackendFactory.java
index 54a9a81..f51d414 100644
--- a/samza-api/src/main/java/org/apache/samza/storage/StateBackendFactory.java
+++ b/samza-api/src/main/java/org/apache/samza/storage/StateBackendFactory.java
@@ -24,18 +24,20 @@ import java.util.concurrent.ExecutorService;
 import org.apache.samza.config.Config;
 import org.apache.samza.context.ContainerContext;
 import org.apache.samza.context.JobContext;
+import org.apache.samza.job.model.ContainerModel;
+import org.apache.samza.job.model.JobModel;
 import org.apache.samza.job.model.TaskModel;
 import org.apache.samza.metrics.MetricsRegistry;
 import org.apache.samza.util.Clock;
 
 
 /**
- * Factory to build the Samza {@link TaskBackupManager}, {@link TaskRestoreManager} and {@link TaskStorageAdmin}
+ * Factory to build the Samza {@link TaskBackupManager}, {@link TaskRestoreManager} and {@link StateBackendAdmin}
  * for a particular state storage backend, which are used to durably backup the Samza task state.
  */
 public interface StateBackendFactory {
   TaskBackupManager getBackupManager(JobContext jobContext,
-      ContainerContext containerContext,
+      ContainerModel containerModel,
       TaskModel taskModel,
       ExecutorService backupExecutor,
       MetricsRegistry taskInstanceMetricsRegistry,
@@ -55,5 +57,5 @@ public interface StateBackendFactory {
       File nonLoggedStoreBaseDir,
       KafkaChangelogRestoreParams kafkaChangelogRestoreParams);
 
-  TaskStorageAdmin getAdmin();
+  StateBackendAdmin getAdmin(JobModel jobModel, Config config);
 }
diff --git a/samza-api/src/main/java/org/apache/samza/storage/blobstore/BlobStoreManager.java b/samza-api/src/main/java/org/apache/samza/storage/blobstore/BlobStoreManager.java
new file mode 100644
index 0000000..ef3f0c6
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/storage/blobstore/BlobStoreManager.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.storage.blobstore;
+
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.concurrent.CompletionStage;
+import org.apache.samza.annotation.InterfaceStability;
+
+
+/**
+ * Provides interface for common blob store operations: GET, PUT and DELETE
+ */
+@InterfaceStability.Unstable
+public interface BlobStoreManager {
+  /**
+   * Initialize underlying blob store client, if necessary.
+   *
+   */
+  void init();
+
+  /**
+   * Non-blocking PUT call to remote blob store with supplied metadata
+   * @param inputStream InputStream to read the file
+   * @param metadata user supplied {@link Metadata} of the request
+   * @return a future containing the blob ID of the uploaded blob if the upload is successful.
+   */
+  CompletionStage<String> put(InputStream inputStream, Metadata metadata);
+
+  /**
+   * Non-blocking GET call to remote blob store
+   * @param id Blob ID of the blob to get
+   * @param outputStream OutputStream to write the downloaded blob
+   * @param metadata User supplied {@link Metadata} of the request
+   * @return A future that completes when all the chunks are downloaded and written successfully to the OutputStream
+   * @throws org.apache.samza.storage.blobstore.exceptions.DeletedException returned future should complete
+   *         exceptionally with DeletedException on failure with the blob already deleted error.
+   */
+  CompletionStage<Void> get(String id, OutputStream outputStream, Metadata metadata);
+
+  /**
+   * Non-blocking call to mark a blob for deletion in the remote blob store
+   * @param id Blob ID of the blob to delete
+   * @param metadata User supplied {@link Metadata} of the request
+   * @return A future that completes when the blob is successfully deleted from the blob store.
+   * @throws org.apache.samza.storage.blobstore.exceptions.DeletedException returned future should complete
+   *         exceptionally with DeletedException on failure with the blob already deleted error. This exception is
+   *         caught and ignored by the caller of the delete method during initial cleanup and SnapshotIndex read.
+   */
+  CompletionStage<Void> delete(String id, Metadata metadata);
+
+  /**
+   * Non-blocking call to remove the Time-To-Live (TTL) for a blob and make it permanent.
+   * @param blobId Blob ID of blob to remove TTL for.
+   * @param metadata User supplied {@link Metadata} of the request
+   * @return a future that completes when the TTL for the blob is removed.
+   * @throws org.apache.samza.storage.blobstore.exceptions.DeletedException returned future should complete
+   *         exceptionally with DeletedException on failure with the blob already deleted error.
+   */
+  CompletionStage<Void> removeTTL(String blobId, Metadata metadata);
+
+  /**
+   * Cleanly close resources like blob store client
+   */
+  void close();
+}
diff --git a/samza-api/src/main/java/org/apache/samza/storage/TaskStorageAdmin.java b/samza-api/src/main/java/org/apache/samza/storage/blobstore/BlobStoreManagerFactory.java
similarity index 64%
copy from samza-api/src/main/java/org/apache/samza/storage/TaskStorageAdmin.java
copy to samza-api/src/main/java/org/apache/samza/storage/blobstore/BlobStoreManagerFactory.java
index 205077b..5a91393 100644
--- a/samza-api/src/main/java/org/apache/samza/storage/TaskStorageAdmin.java
+++ b/samza-api/src/main/java/org/apache/samza/storage/blobstore/BlobStoreManagerFactory.java
@@ -17,14 +17,15 @@
  * under the License.
  */
 
-package org.apache.samza.storage;
+package org.apache.samza.storage.blobstore;
 
-/**
- * Creates and validate resources for the StateBackendFactory
- */
-public interface TaskStorageAdmin {
+import java.util.concurrent.ExecutorService;
+import org.apache.samza.annotation.InterfaceStability;
+import org.apache.samza.config.Config;
 
-  void createResources();
+@InterfaceStability.Unstable
+public interface BlobStoreManagerFactory {
+  BlobStoreManager getBackupBlobStoreManager(Config config, ExecutorService backupExecutor);
 
-  void validateResources();
+  BlobStoreManager getRestoreBlobStoreManager(Config config, ExecutorService restoreExecutor);
 }
diff --git a/samza-api/src/main/java/org/apache/samza/storage/blobstore/Metadata.java b/samza-api/src/main/java/org/apache/samza/storage/blobstore/Metadata.java
new file mode 100644
index 0000000..3423f0b
--- /dev/null
+++ b/samza-api/src/main/java/org/apache/samza/storage/blobstore/Metadata.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.storage.blobstore;
+
+import java.util.Optional;
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+
+
+/**
+ * Metadata associated with every BlobStore request. This class is used to trace a request and to determine the
+ * bucket/container informationof the blob.
+ */
+public class Metadata {
+  public static final String SNAPSHOT_INDEX_PAYLOAD_PATH = "snapshot-index";
+
+  private final String payloadPath;
+  private final long payloadSize;
+  private final String jobName;
+  private final String jobId;
+  private final String taskName;
+  private final String storeName;
+
+  public Metadata(String payloadPath, Optional<Long> payloadSize,
+      String jobName, String jobId, String taskName, String storeName) {
+    this.payloadPath = payloadPath;
+    // Payload size may not be known in advance for requests like getSnapshotIndex, where only blob ID is known. Set -1.
+    this.payloadSize = payloadSize.orElse(-1L);
+    this.jobName = jobName;
+    this.jobId = jobId;
+    this.taskName = taskName;
+    this.storeName = storeName;
+  }
+
+  public String getPayloadPath() {
+    return payloadPath;
+  }
+
+  public long getPayloadSize() {
+    return payloadSize;
+  }
+
+  public String getJobName() {
+    return jobName;
+  }
+
+  public String getJobId() {
+    return jobId;
+  }
+
+  public String getTaskName() {
+    return taskName;
+  }
+
+  public String getStoreName() {
+    return storeName;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+
+    if (!(o instanceof Metadata)) {
+      return false;
+    }
+
+    Metadata that = (Metadata) o;
+
+    return new EqualsBuilder().append(getPayloadPath(), that.getPayloadPath())
+        .append(getPayloadSize(), that.getPayloadSize())
+        .append(getJobName(), that.getJobName())
+        .append(getJobId(), that.getJobId())
+        .append(getTaskName(), that.getTaskName())
+        .append(getStoreName(), that.getStoreName())
+        .isEquals();
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder(17, 37).append(getPayloadPath())
+        .append(getPayloadSize())
+        .append(getJobName())
+        .append(getJobId())
+        .append(getTaskName())
+        .append(getStoreName())
+        .toHashCode();
+  }
+
+  @Override
+  public String toString() {
+    return "Metadata{" + "payloadPath='" + payloadPath + '\'' + ", payloadSize='" + payloadSize + '\''
+        + ", jobName='" + jobName + '\'' + ", jobId='" + jobId + '\'' + ", taskName='" + taskName + '\''
+        + ", storeName='" + storeName + '\'' + '}';
+  }
+}
diff --git a/samza-api/src/main/java/org/apache/samza/storage/TaskStorageAdmin.java b/samza-api/src/main/java/org/apache/samza/storage/blobstore/exceptions/DeletedException.java
similarity index 54%
copy from samza-api/src/main/java/org/apache/samza/storage/TaskStorageAdmin.java
copy to samza-api/src/main/java/org/apache/samza/storage/blobstore/exceptions/DeletedException.java
index 205077b..8fcd230 100644
--- a/samza-api/src/main/java/org/apache/samza/storage/TaskStorageAdmin.java
+++ b/samza-api/src/main/java/org/apache/samza/storage/blobstore/exceptions/DeletedException.java
@@ -17,14 +17,30 @@
  * under the License.
  */
 
-package org.apache.samza.storage;
+package org.apache.samza.storage.blobstore.exceptions;
 
 /**
- * Creates and validate resources for the StateBackendFactory
+ * Future should complete with this exception to indicate that the exception occurred due to the request for an
+ * already deleted blob. This exception is caught and ignored by caller of the DELETE request during initial cleanup
+ * and snapshot index read.
+ *
  */
-public interface TaskStorageAdmin {
+public class DeletedException extends RuntimeException {
+
+  private static final long serialVersionUID = 1L;
+
+  public DeletedException(String message, Throwable cause) {
+    super(message, cause);
+  }
+
+  public DeletedException(String message) {
+    super(message);
+  }
 
-  void createResources();
+  public DeletedException(Throwable cause) {
+    super(cause);
+  }
 
-  void validateResources();
+  public DeletedException() {
+  }
 }
diff --git a/samza-api/src/main/java/org/apache/samza/storage/TaskStorageAdmin.java b/samza-api/src/main/java/org/apache/samza/storage/blobstore/exceptions/RetriableException.java
similarity index 64%
copy from samza-api/src/main/java/org/apache/samza/storage/TaskStorageAdmin.java
copy to samza-api/src/main/java/org/apache/samza/storage/blobstore/exceptions/RetriableException.java
index 205077b..e4ff789 100644
--- a/samza-api/src/main/java/org/apache/samza/storage/TaskStorageAdmin.java
+++ b/samza-api/src/main/java/org/apache/samza/storage/blobstore/exceptions/RetriableException.java
@@ -17,14 +17,26 @@
  * under the License.
  */
 
-package org.apache.samza.storage;
+package org.apache.samza.storage.blobstore.exceptions;
 
-/**
- * Creates and validate resources for the StateBackendFactory
- */
-public interface TaskStorageAdmin {
 
-  void createResources();
+public class RetriableException extends RuntimeException {
+
+  private static final long serialVersionUID = 1L;
+
+  public RetriableException(String message, Throwable cause) {
+    super(message, cause);
+  }
 
-  void validateResources();
+  public RetriableException(String message) {
+    super(message);
+  }
+
+  public RetriableException(Throwable cause) {
+    super(cause);
+  }
+
+  public RetriableException() {
+  }
 }
+
diff --git a/samza-core/src/main/java/org/apache/samza/clustermanager/ClusterBasedJobCoordinator.java b/samza-core/src/main/java/org/apache/samza/clustermanager/ClusterBasedJobCoordinator.java
index 08bcfda..e956413 100644
--- a/samza-core/src/main/java/org/apache/samza/clustermanager/ClusterBasedJobCoordinator.java
+++ b/samza-core/src/main/java/org/apache/samza/clustermanager/ClusterBasedJobCoordinator.java
@@ -59,10 +59,13 @@ import org.apache.samza.metrics.JmxServer;
 import org.apache.samza.metrics.MetricsRegistryMap;
 import org.apache.samza.startpoint.StartpointManager;
 import org.apache.samza.storage.ChangelogStreamManager;
+import org.apache.samza.storage.StateBackendAdmin;
+import org.apache.samza.storage.StateBackendFactory;
 import org.apache.samza.system.StreamMetadataCache;
 import org.apache.samza.system.SystemAdmins;
 import org.apache.samza.system.SystemStream;
 import org.apache.samza.util.DiagnosticsUtil;
+import org.apache.samza.util.ReflectionUtil;
 import org.apache.samza.util.SystemClock;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -267,6 +270,18 @@ public class ClusterBasedJobCoordinator {
       MetadataResourceUtil metadataResourceUtil = new MetadataResourceUtil(jobModel, this.metrics, config);
       metadataResourceUtil.createResources();
 
+      // create all the resources required for state backend factories
+      StorageConfig storageConfig = new StorageConfig(config);
+      storageConfig.getBackupFactories().forEach(stateStorageBackendBackupFactory -> {
+        StateBackendFactory stateBackendFactory =
+            ReflectionUtil.getObj(stateStorageBackendBackupFactory, StateBackendFactory.class);
+        StateBackendAdmin stateBackendAdmin = stateBackendFactory.getAdmin(jobModel, config);
+        // Create resources required for state backend admin
+        stateBackendAdmin.createResources();
+        // Validate resources required for state backend admin
+        stateBackendAdmin.validateResources();
+      });
+
       /*
        * We fanout startpoint if and only if
        *  1. Startpoint is enabled in configuration
diff --git a/samza-api/src/main/java/org/apache/samza/storage/TaskStorageAdmin.java b/samza-core/src/main/java/org/apache/samza/config/BlobStoreConfig.java
similarity index 59%
copy from samza-api/src/main/java/org/apache/samza/storage/TaskStorageAdmin.java
copy to samza-core/src/main/java/org/apache/samza/config/BlobStoreConfig.java
index 205077b..769ba29 100644
--- a/samza-api/src/main/java/org/apache/samza/storage/TaskStorageAdmin.java
+++ b/samza-core/src/main/java/org/apache/samza/config/BlobStoreConfig.java
@@ -17,14 +17,25 @@
  * under the License.
  */
 
-package org.apache.samza.storage;
+package org.apache.samza.config;
 
 /**
- * Creates and validate resources for the StateBackendFactory
+ * Config related helper methods for BlobStore.
  */
-public interface TaskStorageAdmin {
+public class BlobStoreConfig extends MapConfig {
 
-  void createResources();
+  public static final String BLOB_STORE_MANAGER_FACTORY = "blob.store.manager.factory";
+  public static final String BLOB_STORE_ADMIN_FACTORY = "blob.store.admin.factory";
+  public BlobStoreConfig(Config config) {
+    super(config);
+  }
 
-  void validateResources();
+
+  public String getBlobStoreManagerFactory() {
+    return get(BLOB_STORE_MANAGER_FACTORY);
+  }
+
+  public String getBlobStoreAdminFactory() {
+    return get(BLOB_STORE_ADMIN_FACTORY);
+  }
 }
diff --git a/samza-core/src/main/java/org/apache/samza/config/StorageConfig.java b/samza-core/src/main/java/org/apache/samza/config/StorageConfig.java
index 4dd753e..aa8df95 100644
--- a/samza-core/src/main/java/org/apache/samza/config/StorageConfig.java
+++ b/samza-core/src/main/java/org/apache/samza/config/StorageConfig.java
@@ -66,13 +66,15 @@ public class StorageConfig extends MapConfig {
   public static final String CHANGELOG_MIN_COMPACTION_LAG_MS = STORE_PREFIX + "%s.changelog." + MIN_COMPACTION_LAG_MS;
   public static final long DEFAULT_CHANGELOG_MIN_COMPACTION_LAG_MS = TimeUnit.HOURS.toMillis(4);
 
-  public static final String DEFAULT_STATE_BACKEND_FACTORY = "org.apache.samza.storage.KafkaChangelogStateBackendFactory";
-  public static final String STORE_BACKEND_BACKUP_FACTORIES = STORE_PREFIX + "%s.state.backend.backup.factories";
-  public static final List<String> DEFAULT_STATE_BACKEND_BACKUP_FACTORIES = ImmutableList.of(
-      DEFAULT_STATE_BACKEND_FACTORY);
-  public static final String STATE_BACKEND_RESTORE_FACTORY = STORE_PREFIX + "state.restore.backend";
   public static final String INMEMORY_KV_STORAGE_ENGINE_FACTORY =
       "org.apache.samza.storage.kv.inmemory.InMemoryKeyValueStorageEngineFactory";
+  public static final String KAFKA_STATE_BACKEND_FACTORY =
+      "org.apache.samza.storage.KafkaChangelogStateBackendFactory";
+  public static final List<String> DEFAULT_BACKUP_FACTORIES = ImmutableList.of(
+      KAFKA_STATE_BACKEND_FACTORY);
+  public static final String STORE_BACKUP_FACTORIES = STORE_PREFIX + "%s.backup.factories";
+  // TODO BLOCKER dchen make this per store
+  public static final String STORE_RESTORE_FACTORY = STORE_PREFIX + "restore.factory";
 
   static final String CHANGELOG_SYSTEM = "job.changelog.system";
   static final String CHANGELOG_DELETE_RETENTION_MS = STORE_PREFIX + "%s.changelog.delete.retention.ms";
@@ -138,15 +140,6 @@ public class StorageConfig extends MapConfig {
     return Optional.ofNullable(systemStreamRes);
   }
 
-  public List<String> getStoreBackupManagerClassName(String storeName) {
-    List<String> storeBackupManagers = getList(String.format(STORE_BACKEND_BACKUP_FACTORIES, storeName), new ArrayList<>());
-    // For backwards compatibility if the changelog is enabled, we use default kafka backup factory
-    if (storeBackupManagers.isEmpty() && getChangelogStream(storeName).isPresent()) {
-      storeBackupManagers = DEFAULT_STATE_BACKEND_BACKUP_FACTORIES;
-    }
-    return storeBackupManagers;
-  }
-
   public boolean getAccessLogEnabled(String storeName) {
     return getBoolean(String.format(ACCESSLOG_ENABLED, storeName), false);
   }
@@ -265,24 +258,6 @@ public class StorageConfig extends MapConfig {
     return getLong(minCompactLagConfigName, getDefaultChangelogMinCompactionLagMs());
   }
 
-
-  public Set<String> getStateBackendBackupFactories() {
-    return getStoreNames().stream()
-        .flatMap((storeName) -> getStoreBackupManagerClassName(storeName).stream())
-        .collect(Collectors.toSet());
-  }
-
-  public List<String> getBackupStoreNamesForStateBackupFactory(String backendFactoryName) {
-    return getStoreNames().stream()
-        .filter((storeName) -> getStoreBackupManagerClassName(storeName)
-            .contains(backendFactoryName))
-        .collect(Collectors.toList());
-  }
-
-  public String getStateBackendRestoreFactory() {
-    return get(STATE_BACKEND_RESTORE_FACTORY, DEFAULT_STATE_BACKEND_FACTORY);
-  }
-
   /**
    * Helper method to check if a system has a changelog attached to it.
    */
@@ -313,11 +288,44 @@ public class StorageConfig extends MapConfig {
         .count();
   }
 
+  public List<String> getStoreBackupFactory(String storeName) {
+    List<String> storeBackupManagers = getList(String.format(STORE_BACKUP_FACTORIES, storeName), new ArrayList<>());
+    // For backwards compatibility if the changelog is enabled, we use default kafka backup factory
+    if (storeBackupManagers.isEmpty() && getChangelogStream(storeName).isPresent()) {
+      storeBackupManagers = DEFAULT_BACKUP_FACTORIES;
+    }
+    return storeBackupManagers;
+  }
+
+  public Set<String> getBackupFactories() {
+    return getStoreNames().stream()
+        .flatMap((storeName) -> getStoreBackupFactory(storeName).stream())
+        .collect(Collectors.toSet());
+  }
+
+  public String getRestoreFactory() {
+    return get(STORE_RESTORE_FACTORY, KAFKA_STATE_BACKEND_FACTORY);
+  }
+
+  public List<String> getStoresWithBackupFactory(String backendFactoryName) {
+    return getStoreNames().stream()
+        .filter((storeName) -> getStoreBackupFactory(storeName)
+            .contains(backendFactoryName))
+        .collect(Collectors.toList());
+  }
+
+  // TODO BLOCKER dchen update when making restore managers per store
+  public List<String> getStoresWithRestoreFactory(String backendFactoryName) {
+    return getStoreNames().stream()
+        .filter((storeName) -> getRestoreFactory().equals(backendFactoryName))
+        .collect(Collectors.toList());
+  }
+
   /**
    * Helper method to get if logged store dirs should be deleted regardless of their contents.
    * @return
    */
-  public boolean getCleanLoggedStoreDirsOnStart(String storeName) {
+  public boolean cleanLoggedStoreDirsOnStart(String storeName) {
     return getBoolean(String.format(CLEAN_LOGGED_STOREDIRS_ON_START, storeName), false);
   }
 }
diff --git a/samza-core/src/main/java/org/apache/samza/standalone/PassthroughJobCoordinator.java b/samza-core/src/main/java/org/apache/samza/standalone/PassthroughJobCoordinator.java
index cc51732..7bf29fe 100644
--- a/samza-core/src/main/java/org/apache/samza/standalone/PassthroughJobCoordinator.java
+++ b/samza-core/src/main/java/org/apache/samza/standalone/PassthroughJobCoordinator.java
@@ -19,25 +19,27 @@
 package org.apache.samza.standalone;
 
 import com.google.common.collect.ImmutableMap;
+import java.util.Collections;
 import org.apache.samza.config.Config;
 import org.apache.samza.config.JobConfig;
 import org.apache.samza.container.grouper.task.GrouperMetadata;
 import org.apache.samza.container.grouper.task.GrouperMetadataImpl;
 import org.apache.samza.coordinator.JobCoordinator;
+import org.apache.samza.coordinator.JobCoordinatorListener;
 import org.apache.samza.coordinator.JobModelManager;
 import org.apache.samza.coordinator.MetadataResourceUtil;
 import org.apache.samza.job.model.JobModel;
-import org.apache.samza.coordinator.JobCoordinatorListener;
+import org.apache.samza.metrics.MetricsRegistry;
 import org.apache.samza.runtime.LocationId;
 import org.apache.samza.runtime.LocationIdProvider;
 import org.apache.samza.runtime.LocationIdProviderFactory;
-import org.apache.samza.metrics.MetricsRegistry;
 import org.apache.samza.system.StreamMetadataCache;
 import org.apache.samza.system.SystemAdmins;
-import org.apache.samza.util.*;
+import org.apache.samza.util.ReflectionUtil;
+import org.apache.samza.util.SystemClock;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import java.util.Collections;
+
 
 /**
  * Standalone Job Coordinator does not implement any leader elector module or cluster manager
@@ -125,7 +127,9 @@ public class PassthroughJobCoordinator implements JobCoordinator {
     systemAdmins.start();
     try {
       String containerId = Integer.toString(config.getInt(JobConfig.PROCESSOR_ID));
-      GrouperMetadata grouperMetadata = new GrouperMetadataImpl(ImmutableMap.of(String.valueOf(containerId), locationId), Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap());
+      GrouperMetadata grouperMetadata =
+          new GrouperMetadataImpl(ImmutableMap.of(String.valueOf(containerId), locationId), Collections.emptyMap(),
+              Collections.emptyMap(), Collections.emptyMap());
       return JobModelManager.readJobModel(this.config, Collections.emptyMap(), streamMetadataCache, grouperMetadata);
     } finally {
       systemAdmins.stop();
diff --git a/samza-core/src/main/java/org/apache/samza/storage/KafkaChangelogStateBackendFactory.java b/samza-core/src/main/java/org/apache/samza/storage/KafkaChangelogStateBackendFactory.java
index 36ea5b5..54f20da 100644
--- a/samza-core/src/main/java/org/apache/samza/storage/KafkaChangelogStateBackendFactory.java
+++ b/samza-core/src/main/java/org/apache/samza/storage/KafkaChangelogStateBackendFactory.java
@@ -28,13 +28,13 @@ import java.util.Set;
 import java.util.concurrent.ExecutorService;
 import java.util.stream.Collectors;
 import org.apache.commons.collections4.MapUtils;
-import org.apache.samza.SamzaException;
 import org.apache.samza.config.Config;
 import org.apache.samza.config.StorageConfig;
 import org.apache.samza.config.TaskConfig;
 import org.apache.samza.context.ContainerContext;
 import org.apache.samza.context.JobContext;
 import org.apache.samza.job.model.ContainerModel;
+import org.apache.samza.job.model.JobModel;
 import org.apache.samza.job.model.TaskMode;
 import org.apache.samza.job.model.TaskModel;
 import org.apache.samza.metrics.MetricsRegistry;
@@ -46,29 +46,13 @@ import org.apache.samza.system.SystemStreamPartition;
 import org.apache.samza.util.Clock;
 
 
-/**
- * Class used the provide the {@link TaskRestoreManager} and the {@link TaskBackupManager} for the Kafka changelog
- * state backend.
- */
 public class KafkaChangelogStateBackendFactory implements StateBackendFactory {
   private StreamMetadataCache streamCache;
-  /*
-   * This keeps track of the changelog SSPs that are associated with the whole container. This is used so that we can
-   * prefetch the metadata about the all of the changelog SSPs associated with the container whenever we need the
-   * metadata about some of the changelog SSPs.
-   * An example use case is when Samza writes offset files for stores ({@link TaskStorageManager}). Each task is
-   * responsible for its own offset file, but if we can do prefetching, then most tasks will already have cached
-   * metadata by the time they need the offset metadata.
-   * Note: By using all changelog streams to build the sspsToPrefetch, any fetches done for persisted stores will
-   * include the ssps for non-persisted stores, so this is slightly suboptimal. However, this does not increase the
-   * actual number of calls to the {@link SystemAdmin}, and we can decouple this logic from the per-task objects (e.g.
-   * {@link TaskStorageManager}).
-   */
   private SSPMetadataCache sspCache;
 
   @Override
   public TaskBackupManager getBackupManager(JobContext jobContext,
-      ContainerContext containerContext,
+      ContainerModel containerModel,
       TaskModel taskModel,
       ExecutorService backupExecutor,
       MetricsRegistry metricsRegistry,
@@ -101,7 +85,10 @@ public class KafkaChangelogStateBackendFactory implements StateBackendFactory {
       File nonLoggedStoreBaseDir,
       KafkaChangelogRestoreParams kafkaChangelogRestoreParams) {
     Map<String, SystemStream> storeChangelogs = new StorageConfig(config).getStoreChangelogs();
-    Set<SystemStreamPartition> changelogSSPs = getChangelogSSPForContainer(storeChangelogs, containerContext);
+    Set<SystemStreamPartition> changelogSSPs = storeChangelogs.values().stream()
+        .flatMap(ss -> containerContext.getContainerModel().getTasks().values().stream()
+            .map(tm -> new SystemStreamPartition(ss, tm.getChangelogPartition())))
+        .collect(Collectors.toSet());
     // filter out standby store-ssp pairs
     Map<String, SystemStream> filteredStoreChangelogs =
         filterStandbySystemStreams(storeChangelogs, containerContext.getContainerModel());
@@ -152,12 +139,12 @@ public class KafkaChangelogStateBackendFactory implements StateBackendFactory {
   }
 
   @Override
-  public TaskStorageAdmin getAdmin() {
-    throw new SamzaException("getAdmin() method not supported for KafkaStateBackendFactory");
+  //TODO HIGH snjain implement this
+  public StateBackendAdmin getAdmin(JobModel jobModel, Config config) {
+    return new NoOpKafkaChangelogStateBackendAdmin();
   }
 
-  @VisibleForTesting
-  Set<SystemStreamPartition> getChangelogSSPForContainer(Map<String, SystemStream> storeChangelogs,
+  public Set<SystemStreamPartition> getChangelogSSPForContainer(Map<String, SystemStream> storeChangelogs,
       ContainerContext containerContext) {
     return storeChangelogs.values().stream()
         .flatMap(ss -> containerContext.getContainerModel().getTasks().values().stream()
@@ -188,8 +175,7 @@ public class KafkaChangelogStateBackendFactory implements StateBackendFactory {
    * @param ssps SSPs to prefetch
    * @return SSPMetadataCache containing the partition metadata
    */
-  @VisibleForTesting
-  SSPMetadataCache getSspCache(SystemAdmins admins, Clock clock, Set<SystemStreamPartition> ssps) {
+  private SSPMetadataCache getSspCache(SystemAdmins admins, Clock clock, Set<SystemStreamPartition> ssps) {
     if (sspCache == null) {
       sspCache = new SSPMetadataCache(admins, Duration.ofSeconds(5), clock, ssps);
     }
@@ -201,15 +187,37 @@ public class KafkaChangelogStateBackendFactory implements StateBackendFactory {
       ContainerModel containerModel) {
     Map<SystemStreamPartition, String> changelogSSPToStore = new HashMap<>();
     changelogSystemStreams.forEach((storeName, systemStream) ->
-        containerModel.getTasks().forEach((taskName, taskModel) -> {
-          if (!TaskMode.Standby.equals(taskModel.getTaskMode())) {
-            changelogSSPToStore.put(new SystemStreamPartition(systemStream, taskModel.getChangelogPartition()),
-                storeName);
-          }
-        })
+        containerModel.getTasks().forEach((taskName, taskModel) ->
+            changelogSSPToStore.put(new SystemStreamPartition(systemStream, taskModel.getChangelogPartition()), storeName))
     );
+
+    Set<TaskModel> standbyTaskModels = containerModel.getTasks().values().stream()
+        .filter(taskModel -> taskModel.getTaskMode().equals(TaskMode.Standby))
+        .collect(Collectors.toSet());
+
+    // remove all standby task changelog ssps
+    standbyTaskModels.forEach((taskModel) -> {
+      changelogSystemStreams.forEach((storeName, systemStream) -> {
+        SystemStreamPartition ssp = new SystemStreamPartition(systemStream, taskModel.getChangelogPartition());
+        changelogSSPToStore.remove(ssp);
+      });
+    });
+
     // changelogSystemStreams correspond only to active tasks (since those of standby-tasks moved to sideInputs above)
     return MapUtils.invertMap(changelogSSPToStore).entrySet().stream()
         .collect(Collectors.toMap(Map.Entry::getKey, x -> x.getValue().getSystemStream()));
   }
+
+  public class NoOpKafkaChangelogStateBackendAdmin implements StateBackendAdmin {
+
+    @Override
+    public void createResources() {
+      // all the changelog creations are handled by {@link ChangelogStreamManager}
+    }
+
+    @Override
+    public void validateResources() {
+      // all the changelog validations are handled by {@link ChangelogStreamManager}
+    }
+  }
 }
diff --git a/samza-core/src/main/java/org/apache/samza/storage/NonTransactionalStateTaskRestoreManager.java b/samza-core/src/main/java/org/apache/samza/storage/NonTransactionalStateTaskRestoreManager.java
index f648b06..35fa375 100644
--- a/samza-core/src/main/java/org/apache/samza/storage/NonTransactionalStateTaskRestoreManager.java
+++ b/samza-core/src/main/java/org/apache/samza/storage/NonTransactionalStateTaskRestoreManager.java
@@ -156,7 +156,7 @@ class NonTransactionalStateTaskRestoreManager implements TaskRestoreManager {
         LOG.info("Got logged storage partition directory as " + loggedStorePartitionDir.toPath().toString());
 
         // Delete the logged store if it is not valid.
-        if (!isLoggedStoreValid(storeName, loggedStorePartitionDir) || storageConfig.getCleanLoggedStoreDirsOnStart(storeName)) {
+        if (!isLoggedStoreValid(storeName, loggedStorePartitionDir) || storageConfig.cleanLoggedStoreDirsOnStart(storeName)) {
           LOG.info("Deleting logged storage partition directory " + loggedStorePartitionDir.toPath().toString());
           fileUtil.rm(loggedStorePartitionDir);
         } else {
diff --git a/samza-core/src/main/java/org/apache/samza/storage/StorageManagerUtil.java b/samza-core/src/main/java/org/apache/samza/storage/StorageManagerUtil.java
index 91c0c7c..057b248 100644
--- a/samza-core/src/main/java/org/apache/samza/storage/StorageManagerUtil.java
+++ b/samza-core/src/main/java/org/apache/samza/storage/StorageManagerUtil.java
@@ -20,12 +20,11 @@
 package org.apache.samza.storage;
 
 import com.fasterxml.jackson.core.JsonParseException;
+import com.fasterxml.jackson.core.type.TypeReference;
 import com.fasterxml.jackson.databind.JsonMappingException;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.ObjectWriter;
-import com.fasterxml.jackson.core.type.TypeReference;
 import com.google.common.collect.ImmutableMap;
-
 import java.io.File;
 import java.io.IOException;
 import java.nio.file.Files;
@@ -71,17 +70,6 @@ public class StorageManagerUtil {
   private static final CheckpointV2Serde CHECKPOINT_V2_SERDE = new CheckpointV2Serde();
 
   /**
-   * Returns the path for a storage engine to create its checkpoint based on the current checkpoint id.
-   *
-   * @param taskStoreDir directory of the store as returned by {@link #getTaskStoreDir}
-   * @param checkpointId current checkpoint id
-   * @return String denoting the file path of the store with the given checkpoint id
-   */
-  public static String getCheckpointDirPath(File taskStoreDir, CheckpointId checkpointId) {
-    return taskStoreDir.getPath() + "-" + checkpointId.serialize();
-  }
-
-  /**
    * Fetch the starting offset for the input {@link SystemStreamPartition}
    *
    * Note: The method doesn't respect {@link org.apache.samza.config.StreamConfig#CONSUMER_OFFSET_DEFAULT} and
@@ -405,6 +393,17 @@ public class StorageManagerUtil {
     }
   }
 
+  /**
+   * Returns the path for a storage engine to create its checkpoint based on the current checkpoint id.
+   *
+   * @param taskStoreDir directory of the store as returned by {@link #getTaskStoreDir}
+   * @param checkpointId current checkpoint id
+   * @return String denoting the file path of the store with the given checkpoint id
+   */
+  public String getStoreCheckpointDir(File taskStoreDir, CheckpointId checkpointId) {
+    return taskStoreDir.getPath() + "-" + checkpointId.serialize();
+  }
+
   public void restoreCheckpointFiles(File checkpointDir, File storeDir) {
     // the current task store dir should already be deleted for restore
     assert !storeDir.exists();
diff --git a/samza-core/src/main/java/org/apache/samza/storage/StorageRecovery.java b/samza-core/src/main/java/org/apache/samza/storage/StorageRecovery.java
index 0925949..8091e48 100644
--- a/samza-core/src/main/java/org/apache/samza/storage/StorageRecovery.java
+++ b/samza-core/src/main/java/org/apache/samza/storage/StorageRecovery.java
@@ -215,7 +215,7 @@ public class StorageRecovery {
    */
   @SuppressWarnings("rawtypes")
   private void getContainerStorageManagers() {
-    String factoryClass = new StorageConfig(jobConfig).getStateBackendRestoreFactory();
+    String factoryClass = new StorageConfig(jobConfig).getRestoreFactory();
     Clock clock = SystemClock.instance();
     StreamMetadataCache streamMetadataCache = new StreamMetadataCache(systemAdmins, 5000, clock);
     // don't worry about prefetching for this; looks like the tool doesn't flush to offset files anyways
diff --git a/samza-core/src/main/java/org/apache/samza/storage/TaskSideInputStorageManager.java b/samza-core/src/main/java/org/apache/samza/storage/TaskSideInputStorageManager.java
index f407b1a..61d30c3 100644
--- a/samza-core/src/main/java/org/apache/samza/storage/TaskSideInputStorageManager.java
+++ b/samza-core/src/main/java/org/apache/samza/storage/TaskSideInputStorageManager.java
@@ -20,15 +20,6 @@
 package org.apache.samza.storage;
 
 import com.google.common.annotations.VisibleForTesting;
-import org.apache.samza.SamzaException;
-import org.apache.samza.container.TaskName;
-import org.apache.samza.job.model.TaskMode;
-import org.apache.samza.system.SystemStreamPartition;
-import org.apache.samza.util.Clock;
-import org.apache.samza.util.FileUtil;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import java.io.File;
 import java.util.HashMap;
 import java.util.Map;
@@ -37,6 +28,14 @@ import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Function;
 import java.util.stream.Collectors;
+import org.apache.samza.SamzaException;
+import org.apache.samza.container.TaskName;
+import org.apache.samza.job.model.TaskMode;
+import org.apache.samza.system.SystemStreamPartition;
+import org.apache.samza.util.Clock;
+import org.apache.samza.util.FileUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 
 /**
diff --git a/samza-core/src/main/java/org/apache/samza/storage/TaskStorageCommitManager.java b/samza-core/src/main/java/org/apache/samza/storage/TaskStorageCommitManager.java
index cc80a48..09a8807 100644
--- a/samza-core/src/main/java/org/apache/samza/storage/TaskStorageCommitManager.java
+++ b/samza-core/src/main/java/org/apache/samza/storage/TaskStorageCommitManager.java
@@ -116,8 +116,9 @@ public class TaskStorageCommitManager {
     LOG.debug("Flushed all storage engines for taskName: {}, checkpoint id: {}",
         taskName, checkpointId);
 
-    long checkpointStartNs = System.nanoTime();
+
     // Checkpoint all persisted and durable stores
+    long checkpointStartNs = System.nanoTime();
     storageEngines.forEach((storeName, storageEngine) -> {
       if (storageEngine.getStoreProperties().isPersistedToDisk() &&
           storageEngine.getStoreProperties().isDurableStore()) {
@@ -212,7 +213,7 @@ public class TaskStorageCommitManager {
             storageManagerUtil.writeCheckpointV2File(storeDir, checkpointV2);
 
             CheckpointId checkpointId = checkpointV2.getCheckpointId();
-            File checkpointDir = Paths.get(StorageManagerUtil.getCheckpointDirPath(storeDir, checkpointId)).toFile();
+            File checkpointDir = Paths.get(storageManagerUtil.getStoreCheckpointDir(storeDir, checkpointId)).toFile();
             storageManagerUtil.writeCheckpointV2File(checkpointDir, checkpointV2);
           } catch (Exception e) {
             throw new SamzaException(
@@ -334,7 +335,7 @@ public class TaskStorageCommitManager {
             writeChangelogOffsetFile(storeName, changelogSSP, newestOffset, currentStoreDir);
 
             // Write changelog SSP offset to the OFFSET files in the store checkpoint directory
-            File checkpointDir = Paths.get(StorageManagerUtil.getCheckpointDirPath(
+            File checkpointDir = Paths.get(storageManagerUtil.getStoreCheckpointDir(
                 currentStoreDir, kafkaChangelogSSPOffset.getCheckpointId())).toFile();
             writeChangelogOffsetFile(storeName, changelogSSP, newestOffset, checkpointDir);
           } else {
diff --git a/samza-core/src/main/java/org/apache/samza/storage/TransactionalStateTaskRestoreManager.java b/samza-core/src/main/java/org/apache/samza/storage/TransactionalStateTaskRestoreManager.java
index 1329a0d..20f653c 100644
--- a/samza-core/src/main/java/org/apache/samza/storage/TransactionalStateTaskRestoreManager.java
+++ b/samza-core/src/main/java/org/apache/samza/storage/TransactionalStateTaskRestoreManager.java
@@ -24,7 +24,6 @@ import com.google.common.base.Preconditions;
 import com.google.common.collect.ArrayListMultimap;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ListMultimap;
-
 import java.io.File;
 import java.util.Collections;
 import java.util.HashMap;
@@ -287,7 +286,7 @@ public class TransactionalStateTaskRestoreManager implements TaskRestoreManager
 
       // if the clean.store.start config is set, delete current and checkpoint dirs, restore from oldest offset to checkpointed
       if (storageEngine.getStoreProperties().isPersistedToDisk() && new StorageConfig(
-        config).getCleanLoggedStoreDirsOnStart(storeName)) {
+        config).cleanLoggedStoreDirsOnStart(storeName)) {
         File currentDir = storageManagerUtil.getTaskStoreDir(loggedStoreBaseDirectory, storeName, taskName, taskMode);
         LOG.info("Marking current directory: {} for store: {} in task: {} for deletion due to clean.on.container.start config.",
             currentDir, storeName, taskName);
diff --git a/samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreBackupManager.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreBackupManager.java
new file mode 100644
index 0000000..15cc87b
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreBackupManager.java
@@ -0,0 +1,350 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.storage.blobstore;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableMap;
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionStage;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.samza.SamzaException;
+import org.apache.samza.checkpoint.Checkpoint;
+import org.apache.samza.checkpoint.CheckpointId;
+import org.apache.samza.config.Config;
+import org.apache.samza.config.JobConfig;
+import org.apache.samza.config.StorageConfig;
+import org.apache.samza.job.model.ContainerModel;
+import org.apache.samza.job.model.JobModel;
+import org.apache.samza.job.model.TaskModel;
+import org.apache.samza.storage.StorageManagerUtil;
+import org.apache.samza.storage.TaskBackupManager;
+import org.apache.samza.storage.blobstore.diff.DirDiff;
+import org.apache.samza.storage.blobstore.index.DirIndex;
+import org.apache.samza.storage.blobstore.index.SnapshotIndex;
+import org.apache.samza.storage.blobstore.index.SnapshotMetadata;
+import org.apache.samza.storage.blobstore.metrics.BlobStoreBackupManagerMetrics;
+import org.apache.samza.storage.blobstore.util.BlobStoreUtil;
+import org.apache.samza.storage.blobstore.util.DirDiffUtil;
+import org.apache.samza.util.Clock;
+import org.apache.samza.util.FutureUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class BlobStoreBackupManager implements TaskBackupManager {
+  private static final Logger LOG = LoggerFactory.getLogger(BlobStoreBackupManager.class);
+
+  private final JobModel jobModel;
+  private final ExecutorService executor;
+  private final String jobName;
+  private final String jobId;
+  private final ContainerModel containerModel;
+  private final TaskModel taskModel;
+  private final String taskName;
+  private final Config config;
+  private final Clock clock;
+  private final StorageManagerUtil storageManagerUtil;
+  private final List<String> storesToBackup;
+  private final File loggedStoreBaseDir;
+  private final BlobStoreManager blobStoreManager;
+  private final BlobStoreUtil blobStoreUtil;
+  private final BlobStoreBackupManagerMetrics metrics;
+
+  /**
+   * Map of store name to a Pair of blob id of {@link SnapshotIndex} and the corresponding {@link SnapshotIndex} from
+   * last successful task checkpoint or {@link #upload}.
+   *
+   * After {@link #init}, the map reflects the contents of the last completed checkpoint for the task from the previous
+   * deployment, if any.
+   *
+   * During regular processing, this map is updated after each successful {@link #upload} with the blob id of
+   * {@link SnapshotIndex} and the corresponding {@link SnapshotIndex} of the upload.
+   *
+   * The contents of this map are used to calculate the diff for local state between the last and the current checkpoint
+   * during {@link #upload}.
+   *
+   * Since the task commit process guarantees that the async stage of the previous commit is complete before another
+   * commit can start, this future is guaranteed to be complete in the call to {@link #upload} during the next commit.
+   *
+   * This field is non-final, since the future itself is replaced in its entirety after init/upload.
+   * The internal map contents are never directly modified (e.g. using puts). It's volatile to ensure visibility
+   * across threads since the map assignment may happen on a different thread than the one reading the contents.
+   */
+  private volatile CompletableFuture<Map<String, Pair<String, SnapshotIndex>>>
+      prevStoreSnapshotIndexesFuture;
+
+  public BlobStoreBackupManager(JobModel jobModel, ContainerModel containerModel, TaskModel taskModel,
+      ExecutorService backupExecutor, BlobStoreBackupManagerMetrics blobStoreTaskBackupMetrics, Config config,
+      Clock clock, File loggedStoreBaseDir, StorageManagerUtil storageManagerUtil, BlobStoreManager blobStoreManager) {
+    this.jobModel = jobModel;
+    this.jobName = new JobConfig(config).getName().get();
+    this.jobId = new JobConfig(config).getJobId();
+    this.containerModel = containerModel;
+    this.taskModel = taskModel;
+    this.taskName = taskModel.getTaskName().getTaskName();
+    this.executor = backupExecutor;
+    this.config = config;
+    this.clock = clock;
+    this.storageManagerUtil = storageManagerUtil;
+    StorageConfig storageConfig = new StorageConfig(config);
+    this.storesToBackup =
+        storageConfig.getStoresWithBackupFactory(BlobStoreStateBackendFactory.class.getName());
+    this.loggedStoreBaseDir = loggedStoreBaseDir;
+    this.blobStoreManager = blobStoreManager;
+    this.blobStoreUtil = createBlobStoreUtil(blobStoreManager, executor, blobStoreTaskBackupMetrics);
+    this.prevStoreSnapshotIndexesFuture = CompletableFuture.completedFuture(ImmutableMap.of());
+    this.metrics = blobStoreTaskBackupMetrics;
+    metrics.initStoreMetrics(storesToBackup);
+  }
+
+  @Override
+  public void init(Checkpoint checkpoint) {
+    long startTime = System.nanoTime();
+    LOG.debug("Initializing blob store backup manager for task: {}", taskName);
+
+    blobStoreManager.init();
+
+    // Note: blocks the caller thread.
+    // TODO LOW shesharma exclude stores that are no longer configured during init
+    Map<String, Pair<String, SnapshotIndex>> prevStoreSnapshotIndexes =
+        blobStoreUtil.getStoreSnapshotIndexes(jobName, jobId, taskName, checkpoint);
+    this.prevStoreSnapshotIndexesFuture =
+        CompletableFuture.completedFuture(ImmutableMap.copyOf(prevStoreSnapshotIndexes));
+    metrics.initNs.set(System.nanoTime() - startTime);
+  }
+
+  @Override
+  public Map<String, String> snapshot(CheckpointId checkpointId) {
+    // No-op. Stores are flushed and checkpoints are created by commit manager
+    return Collections.emptyMap();
+  }
+
+  @Override
+  public CompletableFuture<Map<String, String>> upload(CheckpointId checkpointId, Map<String, String> storeSCMs) {
+    long uploadStartTime = System.nanoTime();
+
+    // reset gauges for each upload
+    metrics.filesToUpload.getValue().set(0L);
+    metrics.bytesToUpload.getValue().set(0L);
+    metrics.filesUploaded.getValue().set(0L);
+    metrics.bytesUploaded.getValue().set(0L);
+    metrics.filesRemaining.getValue().set(0L);
+    metrics.bytesRemaining.getValue().set(0L);
+    metrics.filesToRetain.getValue().set(0L);
+    metrics.bytesToRetain.getValue().set(0L);
+
+    // This map is used to atomically replace the prevStoreSnapshotIndexesFuture map at the end of the task commit
+    Map<String, CompletableFuture<Pair<String, SnapshotIndex>>>
+        storeToSCMAndSnapshotIndexPairFutures = new HashMap<>();
+    // This map is used to return serialized State Checkpoint Markers to the caller
+    Map<String, CompletableFuture<String>> storeToSerializedSCMFuture = new HashMap<>();
+
+    storesToBackup.forEach((storeName) -> {
+      long storeUploadStartTime = System.nanoTime();
+      try {
+        // metadata for the current store snapshot to upload
+        SnapshotMetadata snapshotMetadata = new SnapshotMetadata(checkpointId, jobName, jobId, taskName, storeName);
+
+        // get the local store dir corresponding to the current checkpointId
+        File storeDir = storageManagerUtil.getTaskStoreDir(loggedStoreBaseDir, storeName,
+            taskModel.getTaskName(), taskModel.getTaskMode());
+        String checkpointDirPath = storageManagerUtil.getStoreCheckpointDir(storeDir, checkpointId);
+        File checkpointDir = new File(checkpointDirPath);
+
+        LOG.debug("Got task: {} store: {} storeDir: {} and checkpointDir: {}",
+            taskName, storeName, storeDir, checkpointDir);
+
+        // guaranteed to be available since a new task commit may not start until the previous one is complete
+        Map<String, Pair<String, SnapshotIndex>> prevStoreSnapshotIndexes =
+            prevStoreSnapshotIndexesFuture.get(0, TimeUnit.MILLISECONDS);
+
+        // get the previous store directory contents
+        DirIndex prevDirIndex;
+
+        if (prevStoreSnapshotIndexes.containsKey(storeName)) {
+          prevDirIndex = prevStoreSnapshotIndexes.get(storeName).getRight().getDirIndex();
+        } else {
+          // no previous SnapshotIndex means that this is the first commit for this store. Create an empty DirIndex.
+          prevDirIndex = new DirIndex(checkpointDir.getName(), Collections.emptyList(), Collections.emptyList(),
+              Collections.emptyList(), Collections.emptyList());
+        }
+
+        long dirDiffStartTime = System.nanoTime();
+        // get the diff between previous and current store directories
+        DirDiff dirDiff = DirDiffUtil.getDirDiff(checkpointDir, prevDirIndex, DirDiffUtil.areSameFile(false));
+        metrics.storeDirDiffNs.get(storeName).update(System.nanoTime() - dirDiffStartTime);
+
+        DirDiff.Stats stats = DirDiff.getStats(dirDiff);
+        updateStoreDiffMetrics(storeName, stats);
+        metrics.filesToUpload.getValue().addAndGet(stats.filesAdded);
+        metrics.bytesToUpload.getValue().addAndGet(stats.bytesAdded);
+        // Note: FilesRemaining metric is set to FilesAdded in the beginning of the current upload and then counted down
+        // for each upload.
+        metrics.filesRemaining.getValue().addAndGet(stats.filesAdded);
+        metrics.bytesRemaining.getValue().addAndGet(stats.bytesAdded);
+        metrics.filesToRetain.getValue().addAndGet(stats.filesRetained);
+        metrics.bytesToRetain.getValue().addAndGet(stats.bytesRetained);
+
+        // upload the diff to the blob store and get the new directory index
+        CompletionStage<DirIndex> dirIndexFuture = blobStoreUtil.putDir(dirDiff, snapshotMetadata);
+
+        CompletionStage<SnapshotIndex> snapshotIndexFuture =
+            dirIndexFuture.thenApplyAsync(dirIndex -> {
+              LOG.trace("Dir upload complete. Returning new SnapshotIndex for task: {} store: {}.", taskName, storeName);
+              Optional<String> prevSnapshotIndexBlobId =
+                  Optional.ofNullable(prevStoreSnapshotIndexes.get(storeName)).map(Pair::getLeft);
+              return new SnapshotIndex(clock.currentTimeMillis(), snapshotMetadata, dirIndex, prevSnapshotIndexBlobId);
+            }, executor);
+
+        // upload the new snapshot index to the blob store and get its blob id
+        CompletionStage<String> snapshotIndexBlobIdFuture =
+            snapshotIndexFuture
+                .thenComposeAsync(si -> {
+                  LOG.trace("Uploading Snapshot index for task: {} store: {}", taskName, storeName);
+                  return blobStoreUtil.putSnapshotIndex(si);
+                }, executor);
+
+        // save store name and it's SnapshotIndex blob id and SnapshotIndex pair. At the end of the upload, atomically
+        // update previous snapshot index map with this.
+        CompletableFuture<Pair<String, SnapshotIndex>> scmAndSnapshotIndexPairFuture =
+            FutureUtil.toFutureOfPair(
+                Pair.of(snapshotIndexBlobIdFuture.toCompletableFuture(), snapshotIndexFuture.toCompletableFuture()));
+
+        scmAndSnapshotIndexPairFuture.whenComplete((res, ex) -> {
+          long uploadTimeNs = System.nanoTime() - storeUploadStartTime;
+          metrics.storeUploadNs.get(storeName).update(uploadTimeNs);
+        });
+
+        storeToSCMAndSnapshotIndexPairFutures.put(storeName, scmAndSnapshotIndexPairFuture);
+        storeToSerializedSCMFuture.put(storeName, snapshotIndexBlobIdFuture.toCompletableFuture());
+      } catch (Exception e) {
+        throw new SamzaException(
+            String.format("Error uploading store snapshot to blob store for task: %s, store: %s, checkpointId: %s",
+                taskName, storeName, checkpointId), e);
+      }
+    });
+
+    // replace the previous storeName to snapshot index mapping with the new mapping.
+    this.prevStoreSnapshotIndexesFuture =
+        FutureUtil.toFutureOfMap(storeToSCMAndSnapshotIndexPairFutures);
+
+    return FutureUtil.toFutureOfMap(storeToSerializedSCMFuture)
+        .whenComplete((res, ex) -> metrics.uploadNs.update(System.nanoTime() - uploadStartTime));
+  }
+
+  /**
+   * Clean up would be called at the end of every commit as well as on a container start/restart.
+   * Clean up involves the following steps:
+   * 1. Remove TTL of the snapshot index blob and for any associated files and sub-dirs marked for retention.
+   * 2. Delete the files/subdirs marked for deletion in the snapshot index.
+   * 3. Delete the remote {@link SnapshotIndex} blob for the previous checkpoint.
+   * @param checkpointId the {@link CheckpointId} of the last successfully committed checkpoint.
+   * @param storeSCMs store name to state checkpoint markers for the last successfully committed checkpoint
+   */
+  @Override
+  public CompletableFuture<Void> cleanUp(CheckpointId checkpointId, Map<String, String> storeSCMs) {
+    long startTime = System.nanoTime();
+    List<CompletionStage<Void>> removeTTLFutures = new ArrayList<>();
+    List<CompletionStage<Void>> cleanupRemoteSnapshotFutures = new ArrayList<>();
+    List<CompletionStage<Void>> removePrevRemoteSnapshotFutures = new ArrayList<>();
+
+    // SCM, in case of blob store backup and restore, is just the blob id of SnapshotIndex representing the remote snapshot
+    storeSCMs.forEach((storeName, snapshotIndexBlobId) -> {
+      // Only perform cleanup for stores configured with BlobStore State Backend Factory
+      if (storesToBackup.contains(storeName)) {
+        Metadata requestMetadata =
+            new Metadata(Metadata.SNAPSHOT_INDEX_PAYLOAD_PATH, Optional.empty(), jobName, jobId, taskName, storeName);
+        CompletionStage<SnapshotIndex> snapshotIndexFuture =
+            blobStoreUtil.getSnapshotIndex(snapshotIndexBlobId, requestMetadata);
+
+        // 1. remove TTL of index blob and all of its files and sub-dirs marked for retention
+        CompletionStage<Void> removeTTLFuture =
+            snapshotIndexFuture.thenComposeAsync(snapshotIndex -> {
+              LOG.debug("Removing TTL for index blob: {} and all of its files and sub-dirs for task: {} store :{}",
+                  snapshotIndexBlobId, taskName, storeName);
+              return blobStoreUtil.removeTTL(snapshotIndexBlobId, snapshotIndex, requestMetadata);
+            }, executor);
+        removeTTLFutures.add(removeTTLFuture);
+
+        // 2. delete the files/subdirs marked for deletion in the snapshot index.
+        CompletionStage<Void> cleanupRemoteSnapshotFuture =
+            snapshotIndexFuture.thenComposeAsync(snapshotIndex -> {
+              LOG.debug("Deleting files and dirs to remove for current index blob: {} for task: {} store: {}",
+                  snapshotIndexBlobId, taskName, storeName);
+              return blobStoreUtil.cleanUpDir(snapshotIndex.getDirIndex(), requestMetadata);
+            }, executor);
+
+        cleanupRemoteSnapshotFutures.add(cleanupRemoteSnapshotFuture);
+
+        // 3. delete the remote {@link SnapshotIndex} blob for the previous checkpoint.
+        CompletionStage<Void> removePrevRemoteSnapshotFuture =
+            snapshotIndexFuture.thenComposeAsync(snapshotIndex -> {
+              if (snapshotIndex.getPrevSnapshotIndexBlobId().isPresent()) {
+                String blobId = snapshotIndex.getPrevSnapshotIndexBlobId().get();
+                LOG.debug("Removing previous snapshot index blob: {} from blob store for task: {} store: {}.",
+                    blobId, taskName, storeName);
+                return blobStoreUtil.deleteSnapshotIndexBlob(blobId, requestMetadata);
+              } else {
+                // complete future immediately. There are no previous snapshots index blobs to delete.
+                return CompletableFuture.completedFuture(null);
+              }
+            }, executor);
+        removePrevRemoteSnapshotFutures.add(removePrevRemoteSnapshotFuture);
+      }
+    });
+
+    return FutureUtil.allOf(removeTTLFutures, cleanupRemoteSnapshotFutures, removePrevRemoteSnapshotFutures)
+        .whenComplete((res, ex) -> metrics.cleanupNs.update(System.nanoTime() - startTime));
+  }
+
+  @Override
+  public void close() {
+    blobStoreManager.close();
+  }
+
+  @VisibleForTesting
+  protected BlobStoreUtil createBlobStoreUtil(BlobStoreManager blobStoreManager, ExecutorService executor,
+      BlobStoreBackupManagerMetrics metrics) {
+    return new BlobStoreUtil(blobStoreManager, executor, metrics, null);
+  }
+
+  private void updateStoreDiffMetrics(String storeName, DirDiff.Stats stats) {
+    metrics.storeFilesToUpload.get(storeName).set((long) stats.filesAdded);
+    metrics.storeFilesToRetain.get(storeName).set((long) stats.filesRetained);
+    metrics.storeFilesToRemove.get(storeName).set((long) stats.filesRemoved);
+
+    metrics.storeSubDirsToUpload.get(storeName).set((long) stats.subDirsAdded);
+    metrics.storeSubDirsToRetain.get(storeName).set((long) stats.subDirsRetained);
+    metrics.storeSubDirsToRemove.get(storeName).set((long) stats.subDirsRemoved);
+
+    metrics.storeBytesToUpload.get(storeName).set(stats.bytesAdded);
+    metrics.storeBytesToRetain.get(storeName).set(stats.bytesRetained);
+    metrics.storeBytesToRemove.get(storeName).set(stats.bytesRemoved);
+  }
+}
diff --git a/samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreRestoreManager.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreRestoreManager.java
new file mode 100644
index 0000000..052e49d
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreRestoreManager.java
@@ -0,0 +1,369 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.storage.blobstore;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableSet;
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionStage;
+import java.util.concurrent.ExecutorService;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.samza.SamzaException;
+import org.apache.samza.checkpoint.Checkpoint;
+import org.apache.samza.checkpoint.CheckpointId;
+import org.apache.samza.config.BlobStoreConfig;
+import org.apache.samza.config.Config;
+import org.apache.samza.config.JobConfig;
+import org.apache.samza.config.StorageConfig;
+import org.apache.samza.container.TaskName;
+import org.apache.samza.job.model.TaskMode;
+import org.apache.samza.job.model.TaskModel;
+import org.apache.samza.storage.StorageManagerUtil;
+import org.apache.samza.storage.TaskRestoreManager;
+import org.apache.samza.storage.blobstore.index.DirIndex;
+import org.apache.samza.storage.blobstore.index.SnapshotIndex;
+import org.apache.samza.storage.blobstore.metrics.BlobStoreRestoreManagerMetrics;
+import org.apache.samza.storage.blobstore.util.BlobStoreUtil;
+import org.apache.samza.storage.blobstore.util.DirDiffUtil;
+import org.apache.samza.util.FileUtil;
+import org.apache.samza.util.FutureUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class BlobStoreRestoreManager implements TaskRestoreManager {
+  private static final Logger LOG = LoggerFactory.getLogger(BlobStoreRestoreManager.class);
+  // when checking if checkpoint dir is the same as remote snapshot, exclude the "OFFSET" family of files files
+  // that are written to the checkpoint dir after the remote upload is complete as part of
+  // TaskStorageCommitManager#writeCheckpointToStoreDirectories.
+  private static final Set<String> FILES_TO_IGNORE = ImmutableSet.of(
+      StorageManagerUtil.OFFSET_FILE_NAME_LEGACY,
+      StorageManagerUtil.OFFSET_FILE_NAME_NEW,
+      StorageManagerUtil.SIDE_INPUT_OFFSET_FILE_NAME_LEGACY,
+      StorageManagerUtil.CHECKPOINT_FILE_NAME);
+
+  private final TaskModel taskModel;
+  private final String jobName;
+  private final String jobId;
+  private final ExecutorService executor;
+  private final Config config;
+  private final StorageConfig storageConfig;
+  private final BlobStoreConfig blobStoreConfig;
+  private final StorageManagerUtil storageManagerUtil;
+  private final BlobStoreUtil blobStoreUtil;
+  private final DirDiffUtil dirDiffUtil;
+  private final File loggedBaseDir;
+  private final File nonLoggedBaseDir;
+  private final String taskName;
+  private final List<String> storesToRestore;
+  private final BlobStoreRestoreManagerMetrics metrics;
+
+  private BlobStoreManager blobStoreManager;
+
+  /**
+   * Map of store name and Pair of blob id of SnapshotIndex and the corresponding SnapshotIndex from last successful
+   * task checkpoint
+   */
+  private Map<String, Pair<String, SnapshotIndex>> prevStoreSnapshotIndexes;
+
+  public BlobStoreRestoreManager(TaskModel taskModel, ExecutorService restoreExecutor,
+      BlobStoreRestoreManagerMetrics metrics, Config config, File loggedBaseDir, File nonLoggedBaseDir,
+      StorageManagerUtil storageManagerUtil, BlobStoreManager blobStoreManager) {
+    this.taskModel = taskModel;
+    this.jobName = new JobConfig(config).getName().get();
+    this.jobId = new JobConfig(config).getJobId();
+    this.executor = restoreExecutor; // TODO BLOCKER dchen1 dont block on restore executor
+    this.config = config;
+    this.storageConfig = new StorageConfig(config);
+    this.blobStoreConfig = new BlobStoreConfig(config);
+    this.storageManagerUtil = storageManagerUtil;
+    this.blobStoreManager = blobStoreManager;
+    this.blobStoreUtil = createBlobStoreUtil(blobStoreManager, executor, metrics);
+    this.dirDiffUtil = new DirDiffUtil();
+    this.prevStoreSnapshotIndexes = new HashMap<>();
+    this.loggedBaseDir = loggedBaseDir;
+    this.nonLoggedBaseDir = nonLoggedBaseDir;
+    this.taskName = taskModel.getTaskName().getTaskName();
+    StorageConfig storageConfig = new StorageConfig(config);
+    this.storesToRestore =
+        storageConfig.getStoresWithRestoreFactory(BlobStoreStateBackendFactory.class.getName());
+    this.metrics = metrics;
+  }
+
+  @Override
+  public void init(Checkpoint checkpoint) {
+    long startTime = System.nanoTime();
+    LOG.debug("Initializing blob store restore manager for task: {}", taskName);
+
+    blobStoreManager.init();
+
+    // get previous SCMs from checkpoint
+    prevStoreSnapshotIndexes = blobStoreUtil.getStoreSnapshotIndexes(jobName, jobId, taskName, checkpoint);
+    metrics.getSnapshotIndexNs.set(System.nanoTime() - startTime);
+    LOG.trace("Found previous snapshot index during blob store restore manager init for task: {} to be: {}",
+        taskName, prevStoreSnapshotIndexes);
+
+    metrics.initStoreMetrics(storesToRestore);
+
+    // Note: blocks the caller thread.
+    deleteUnusedStoresFromBlobStore(jobName, jobId, taskName, storageConfig, blobStoreConfig, prevStoreSnapshotIndexes,
+        blobStoreUtil, executor);
+    metrics.initNs.set(System.nanoTime() - startTime);
+  }
+
+  /**
+   * Restore state from checkpoints and state snapshots.
+   * State restore is performed by first retrieving the SnapshotIndex of the previous commit for every store from the
+   * prevStoreSnapshotIndexes map. Local store is deleted to perform a restore from local checkpoint directory or remote
+   * directory. If no local state checkpoint directory is found, or if the local checkpoint directory is different from
+   * the remote snapshot, local checkpoint directory is deleted and a restore from the remote store is done by
+   * downloading the state asynchronously and in parallel.
+   *
+   */
+  @Override
+  public void restore() {
+    restoreStores(jobName, jobId, taskModel.getTaskName(), storesToRestore, prevStoreSnapshotIndexes, loggedBaseDir,
+        storageConfig, metrics, storageManagerUtil, blobStoreUtil, dirDiffUtil, executor);
+  }
+
+  @Override
+  public void close() {
+    blobStoreManager.close();
+  }
+
+  @VisibleForTesting
+  protected BlobStoreUtil createBlobStoreUtil(BlobStoreManager blobStoreManager, ExecutorService executor,
+      BlobStoreRestoreManagerMetrics metrics) {
+    return new BlobStoreUtil(blobStoreManager, executor, null, metrics);
+  }
+
+  /**
+   * Deletes blob store contents for stores that were present in the last checkpoint but are either no longer
+   * present in job configs (removed by user since last deploymetn) or are no longer configured to be backed
+   * up using blob stores.
+   *
+   * This method blocks until all the necessary store contents and snapshot index blobs have been marked for deletion.
+   */
+  @VisibleForTesting
+  static void deleteUnusedStoresFromBlobStore(String jobName, String jobId, String taskName, StorageConfig storageConfig,
+      BlobStoreConfig blobStoreConfig, Map<String, Pair<String, SnapshotIndex>> initialStoreSnapshotIndexes,
+      BlobStoreUtil blobStoreUtil, ExecutorService executor) {
+
+    List<String> storesToBackup =
+        storageConfig.getStoresWithBackupFactory(BlobStoreStateBackendFactory.class.getName());
+    List<String> storesToRestore =
+        storageConfig.getStoresWithRestoreFactory(BlobStoreStateBackendFactory.class.getName());
+
+    List<CompletionStage<Void>> storeDeletionFutures = new ArrayList<>();
+    initialStoreSnapshotIndexes.forEach((storeName, scmAndSnapshotIndex) -> {
+      if (!storesToBackup.contains(storeName) && !storesToRestore.contains(storeName)) {
+        LOG.debug("Removing task: {} store: {} from blob store. It is either no longer used, " +
+            "or is no longer configured to be backed up or restored with blob store.", taskName, storeName);
+        DirIndex dirIndex = scmAndSnapshotIndex.getRight().getDirIndex();
+        Metadata requestMetadata =
+            new Metadata(Metadata.SNAPSHOT_INDEX_PAYLOAD_PATH, Optional.empty(), jobName, jobId, taskName, storeName);
+        CompletionStage<Void> storeDeletionFuture =
+            blobStoreUtil.cleanUpDir(dirIndex, requestMetadata) // delete files and sub-dirs previously marked for removal
+                .thenComposeAsync(v ->
+                    blobStoreUtil.deleteDir(dirIndex, requestMetadata), executor) // deleted files and dirs still present
+                .thenComposeAsync(v -> blobStoreUtil.deleteSnapshotIndexBlob(
+                    scmAndSnapshotIndex.getLeft(), requestMetadata),
+                    executor); // delete the snapshot index blob
+        storeDeletionFutures.add(storeDeletionFuture);
+      }
+    });
+
+    FutureUtil.allOf(storeDeletionFutures).join();
+  }
+
+  /**
+   * Restores all eligible stores in the task.
+   */
+  @VisibleForTesting
+  static void restoreStores(String jobName, String jobId, TaskName taskName, List<String> storesToRestore,
+      Map<String, Pair<String, SnapshotIndex>> prevStoreSnapshotIndexes,
+      File loggedBaseDir, StorageConfig storageConfig, BlobStoreRestoreManagerMetrics metrics,
+      StorageManagerUtil storageManagerUtil, BlobStoreUtil blobStoreUtil, DirDiffUtil dirDiffUtil,
+      ExecutorService executor) {
+    long restoreStartTime = System.nanoTime();
+    List<CompletionStage<Void>> restoreFutures = new ArrayList<>();
+
+    LOG.debug("Starting restore for task: {} stores: {}", taskName, storesToRestore);
+    storesToRestore.forEach(storeName -> {
+      if (!prevStoreSnapshotIndexes.containsKey(storeName)) {
+        LOG.info("No checkpointed snapshot index found for task: {} store: {}. Skipping restore.", taskName, storeName);
+        // TODO HIGH shesharm what should we do with the local state already present on disk, if any?
+        // E.g. this will be the case if user changes a store from changelog based backup and restore to
+        // blob store based backup and restore, both at the same time.
+        return;
+      }
+
+      Pair<String, SnapshotIndex> scmAndSnapshotIndex = prevStoreSnapshotIndexes.get(storeName);
+
+      long storeRestoreStartTime = System.nanoTime();
+      SnapshotIndex snapshotIndex = scmAndSnapshotIndex.getRight();
+      DirIndex dirIndex = snapshotIndex.getDirIndex();
+
+      DirIndex.Stats stats = DirIndex.getStats(dirIndex);
+      metrics.filesToRestore.getValue().addAndGet(stats.filesPresent);
+      metrics.bytesToRestore.getValue().addAndGet(stats.bytesPresent);
+      metrics.filesRemaining.getValue().addAndGet(stats.filesPresent);
+      metrics.bytesRemaining.getValue().addAndGet(stats.bytesPresent);
+
+      CheckpointId checkpointId = snapshotIndex.getSnapshotMetadata().getCheckpointId();
+      File storeDir = storageManagerUtil.getTaskStoreDir(loggedBaseDir, storeName, taskName, TaskMode.Active);
+      Path storeCheckpointDir = Paths.get(storageManagerUtil.getStoreCheckpointDir(storeDir, checkpointId));
+      LOG.trace("Got task: {} store: {} local store directory: {} and local store checkpoint directory: {}",
+          taskName, storeName, storeDir, storeCheckpointDir);
+
+      // we always delete the store dir to preserve transactional state guarantees.
+      try {
+        LOG.debug("Deleting local store directory: {}. Will be restored from local store checkpoint directory " +
+            "or remote snapshot.", storeDir);
+        FileUtils.deleteDirectory(storeDir);
+      } catch (IOException e) {
+        throw new SamzaException(String.format("Error deleting store directory: %s", storeDir), e);
+      }
+
+      boolean shouldRestore = shouldRestore(taskName.getTaskName(), storeName, dirIndex,
+          storeCheckpointDir, storageConfig, dirDiffUtil);
+
+      if (shouldRestore) { // restore the store from the remote blob store
+        // delete all store checkpoint directories. if we only delete the store directory and don't
+        // delete the checkpoint directories, the store size on disk will grow to 2x after restore
+        // until the first commit is completed and older checkpoint dirs are deleted. This is
+        // because the hard-linked checkpoint dir files will no longer be de-duped with the
+        // now-deleted main store directory contents and will take up additional space of their
+        // own during the restore.
+        deleteCheckpointDirs(taskName, storeName, loggedBaseDir, storageManagerUtil);
+
+        metrics.storePreRestoreNs.get(storeName).set(System.nanoTime() - storeRestoreStartTime);
+        enqueueRestore(jobName, jobId, taskName.toString(), storeName, storeDir, dirIndex, storeRestoreStartTime,
+            restoreFutures, blobStoreUtil, dirDiffUtil, metrics, executor);
+      } else {
+        LOG.debug("Renaming store checkpoint directory: {} to store directory: {} since its contents are identical " +
+            "to the remote snapshot.", storeCheckpointDir, storeDir);
+        // atomically rename the checkpoint dir to the store dir
+        new FileUtil().move(storeCheckpointDir.toFile(), storeDir);
+
+        // delete any other checkpoint dirs.
+        deleteCheckpointDirs(taskName, storeName, loggedBaseDir, storageManagerUtil);
+      }
+    });
+
+    // wait for all restores to finish
+    FutureUtil.allOf(restoreFutures).whenComplete((res, ex) -> {
+      LOG.info("Restore completed for task: {} stores", taskName);
+      metrics.restoreNs.set(System.nanoTime() - restoreStartTime);
+    }).join(); // TODO BLOCKER dchen1 make non-blocking.
+  }
+
+  /**
+   * Determines if the store needs to be restored from remote snapshot based on local and remote state.
+   */
+  @VisibleForTesting
+  static boolean shouldRestore(String taskName, String storeName, DirIndex dirIndex,
+      Path storeCheckpointDir, StorageConfig storageConfig, DirDiffUtil dirDiffUtil) {
+    // if a store checkpoint directory exists for the last successful task checkpoint, try to use it.
+    boolean restoreStore;
+    if (Files.exists(storeCheckpointDir)) {
+      if (storageConfig.cleanLoggedStoreDirsOnStart(storeName)) {
+        LOG.debug("Restoring task: {} store: {} from remote snapshot since the store is configured to be " +
+            "restored on each restart.", taskName, storeName);
+        restoreStore = true;
+      } else if (dirDiffUtil.areSameDir(FILES_TO_IGNORE, false).test(storeCheckpointDir.toFile(), dirIndex)) {
+        restoreStore = false; // no restore required for this store.
+      } else {
+        // we don't optimize for the case when the local host doesn't contain the most recent store checkpoint
+        // directory but contains an older checkpoint directory which could have partial overlap with the remote
+        // snapshot. we also don't try to optimize for any edge cases where the most recent checkpoint directory
+        // contents could be partially different than the remote store (afaik, there is no known valid scenario
+        // where this could happen right now, except for the offset file handling above).
+        // it's simpler and fast enough for now to restore the entire store instead.
+
+        LOG.error("Local store checkpoint directory: {} contents are not the same as the remote snapshot. " +
+            "Queuing for restore from remote snapshot.", storeCheckpointDir);
+        restoreStore = true;
+      }
+    } else { // did not find last checkpoint dir, restore the store from the remote blob store
+      LOG.debug("No local store checkpoint directory found at: {}. " +
+          "Queuing for restore from remote snapshot.", storeCheckpointDir);
+      restoreStore = true;
+    }
+
+    return restoreStore;
+  }
+
+  /**
+   * Starts the restore for the store, enqueuing all restore-completion futures into {@param restoreFutures}.
+   */
+  @VisibleForTesting
+  static void enqueueRestore(String jobName, String jobId, String taskName, String storeName, File storeDir, DirIndex dirIndex,
+      long storeRestoreStartTime, List<CompletionStage<Void>> restoreFutures, BlobStoreUtil blobStoreUtil,
+      DirDiffUtil dirDiffUtil, BlobStoreRestoreManagerMetrics metrics, ExecutorService executor) {
+
+    Metadata requestMetadata = new Metadata(storeDir.getAbsolutePath(), Optional.empty(), jobName, jobId, taskName, storeName);
+    CompletableFuture<Void> restoreFuture =
+        blobStoreUtil.restoreDir(storeDir, dirIndex, requestMetadata).thenRunAsync(() -> {
+          metrics.storeRestoreNs.get(storeName).set(System.nanoTime() - storeRestoreStartTime);
+
+          long postRestoreStartTime = System.nanoTime();
+          LOG.trace("Comparing restored store directory: {} and remote directory to verify restore.", storeDir);
+          if (!dirDiffUtil.areSameDir(FILES_TO_IGNORE, true).test(storeDir, dirIndex)) {
+            metrics.storePostRestoreNs.get(storeName).set(System.nanoTime() - postRestoreStartTime);
+            throw new SamzaException(
+                String.format("Restored store directory: %s contents " + "are not the same as the remote snapshot.",
+                    storeDir.getAbsolutePath()));
+          } else {
+            metrics.storePostRestoreNs.get(storeName).set(System.nanoTime() - postRestoreStartTime);
+            LOG.info("Restore from remote snapshot completed for store: {}", storeDir);
+          }
+        }, executor);
+
+    restoreFutures.add(restoreFuture);
+  }
+
+  private static void deleteCheckpointDirs(TaskName taskName, String storeName, File loggedBaseDir, StorageManagerUtil storageManagerUtil) {
+    try {
+      List<File> checkpointDirs = storageManagerUtil.getTaskStoreCheckpointDirs(
+          loggedBaseDir, storeName, taskName, TaskMode.Active);
+      for (File checkpointDir: checkpointDirs) {
+        LOG.debug("Deleting local store checkpoint directory: {} before restore.", checkpointDir);
+        FileUtils.deleteDirectory(checkpointDir);
+      }
+    } catch (Exception e) {
+      throw new SamzaException(
+          String.format("Error deleting checkpoint directory for task: %s store: %s.",
+              taskName, storeName), e);
+    }
+  }
+}
diff --git a/samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreStateBackendFactory.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreStateBackendFactory.java
new file mode 100644
index 0000000..23482c7
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreStateBackendFactory.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.storage.blobstore;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.util.concurrent.ExecutorService;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.samza.config.BlobStoreConfig;
+import org.apache.samza.config.Config;
+import org.apache.samza.context.ContainerContext;
+import org.apache.samza.context.JobContext;
+import org.apache.samza.job.model.ContainerModel;
+import org.apache.samza.job.model.JobModel;
+import org.apache.samza.job.model.TaskModel;
+import org.apache.samza.metrics.MetricsRegistry;
+import org.apache.samza.storage.BlobStoreAdminFactory;
+import org.apache.samza.storage.KafkaChangelogRestoreParams;
+import org.apache.samza.storage.StateBackendAdmin;
+import org.apache.samza.storage.StateBackendFactory;
+import org.apache.samza.storage.StorageManagerUtil;
+import org.apache.samza.storage.TaskBackupManager;
+import org.apache.samza.storage.TaskRestoreManager;
+import org.apache.samza.storage.blobstore.metrics.BlobStoreBackupManagerMetrics;
+import org.apache.samza.storage.blobstore.metrics.BlobStoreRestoreManagerMetrics;
+import org.apache.samza.util.Clock;
+import org.apache.samza.util.ReflectionUtil;
+
+
+public class BlobStoreStateBackendFactory implements StateBackendFactory {
+  @Override
+  public TaskBackupManager getBackupManager(
+      JobContext jobContext,
+      ContainerModel containerModel,
+      TaskModel taskModel,
+      ExecutorService backupExecutor,
+      MetricsRegistry metricsRegistry,
+      Config config,
+      Clock clock,
+      File loggedStoreBaseDir,
+      File nonLoggedStoreBaseDir) {
+    BlobStoreConfig blobStoreConfig = new BlobStoreConfig(config);
+    String blobStoreManagerFactory = blobStoreConfig.getBlobStoreManagerFactory();
+    Preconditions.checkState(StringUtils.isNotBlank(blobStoreManagerFactory));
+    BlobStoreManagerFactory factory = ReflectionUtil.getObj(blobStoreManagerFactory, BlobStoreManagerFactory.class);
+    BlobStoreManager blobStoreManager = factory.getBackupBlobStoreManager(config, backupExecutor);
+    BlobStoreBackupManagerMetrics metrics = new BlobStoreBackupManagerMetrics(metricsRegistry);
+    return new BlobStoreBackupManager(jobContext.getJobModel(), containerModel, taskModel, backupExecutor,
+        metrics, config, clock, loggedStoreBaseDir, new StorageManagerUtil(), blobStoreManager);
+  }
+
+  @Override
+  public TaskRestoreManager getRestoreManager(
+      JobContext jobContext,
+      ContainerContext containerContext,
+      TaskModel taskModel,
+      ExecutorService restoreExecutor,
+      MetricsRegistry metricsRegistry,
+      Config config,
+      Clock clock,
+      File loggedStoreBaseDir,
+      File nonLoggedStoreBaseDir,
+      KafkaChangelogRestoreParams kafkaChangelogRestoreParams) {
+    BlobStoreConfig blobStoreConfig = new BlobStoreConfig(config);
+    String blobStoreManagerFactory = blobStoreConfig.getBlobStoreManagerFactory();
+    Preconditions.checkState(StringUtils.isNotBlank(blobStoreManagerFactory));
+    BlobStoreManagerFactory factory = ReflectionUtil.getObj(blobStoreManagerFactory, BlobStoreManagerFactory.class);
+    BlobStoreManager blobStoreManager = factory.getRestoreBlobStoreManager(config, restoreExecutor);
+    BlobStoreRestoreManagerMetrics metrics = new BlobStoreRestoreManagerMetrics(metricsRegistry);
+    return new BlobStoreRestoreManager(taskModel, restoreExecutor, metrics, config, loggedStoreBaseDir,
+        nonLoggedStoreBaseDir, new StorageManagerUtil(), blobStoreManager);
+  }
+
+  @Override
+  public StateBackendAdmin getAdmin(JobModel jobModel, Config config) {
+    BlobStoreConfig blobStoreConfig = new BlobStoreConfig(config);
+    String stateBackendAdminFactory = blobStoreConfig.getBlobStoreAdminFactory();
+    BlobStoreAdminFactory factory = ReflectionUtil.getObj(stateBackendAdminFactory, BlobStoreAdminFactory.class);
+    return factory.getStateBackendAdmin(config, jobModel);
+  }
+}
\ No newline at end of file
diff --git a/samza-core/src/main/java/org/apache/samza/storage/blobstore/diff/DirDiff.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/diff/DirDiff.java
new file mode 100644
index 0000000..76fdb08
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/diff/DirDiff.java
@@ -0,0 +1,244 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.storage.blobstore.diff;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Sets;
+import java.io.File;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.samza.storage.blobstore.index.DirIndex;
+import org.apache.samza.storage.blobstore.index.FileIndex;
+
+/**
+ * Representation of the diff between a local directory and a remote directory contents.
+ */
+public class DirDiff {
+
+  private final String dirName;
+
+  /**
+   * New files in this directory that needs to be uploaded to the blob store.
+   */
+  private final List<File> filesAdded;
+
+  /**
+   * Files that have already been uploaded to the blob store in a previous snapshot and haven't changed.
+   */
+  private final List<FileIndex> filesRetained;
+
+  /**
+   * Files that have already been uploaded to the blob store in a previous snapshot and need to be removed.
+   */
+  private final List<FileIndex> filesRemoved;
+
+  /**
+   * Subdirectories of this directory that are not already present in the previous snapshot and all of their contents
+   * need to be recursively added.
+   */
+  private final List<DirDiff> subDirsAdded;
+
+  /**
+   * Subdirectories of this directory that are already present in the previous snapshot, but whose contents
+   * may have changed and may need to be recursively added or removed.
+   */
+  private final List<DirDiff> subDirsRetained;
+
+  /**
+   * Subdirectories that are already present in the previous snapshot, but don't exist in the local snapshot,
+   * and hence all of their contents need to be recursively removed.
+   */
+  private final List<DirIndex> subDirsRemoved;
+
+  public DirDiff(String dirName,
+      List<File> filesAdded, List<FileIndex> filesRetained, List<FileIndex> filesRemoved,
+      List<DirDiff> subDirsAdded, List<DirDiff> subDirsRetained, List<DirIndex> subDirsRemoved) {
+    Preconditions.checkNotNull(dirName); // may be empty for root dirs
+    Preconditions.checkNotNull(filesAdded);
+    Preconditions.checkNotNull(filesRetained);
+    Preconditions.checkNotNull(filesRemoved);
+    Preconditions.checkNotNull(subDirsAdded);
+    Preconditions.checkNotNull(subDirsRetained);
+    Preconditions.checkNotNull(subDirsRemoved);
+
+    // validate that a file is not present in multiple lists
+    Set<String> addedFilesSet = filesAdded.stream().map(File::getName).collect(Collectors.toSet());
+    Set<String> retainedFilesSet = filesRetained.stream().map(FileIndex::getFileName).collect(Collectors.toSet());
+    Set<String> removedFilesSet = filesRemoved.stream().map(FileIndex::getFileName).collect(Collectors.toSet());
+    Sets.SetView<String> addedAndRetainedFilesSet = Sets.intersection(addedFilesSet, retainedFilesSet);
+    Preconditions.checkState(addedAndRetainedFilesSet.isEmpty(),
+        String.format("Files present in both added and retained sets: %s", addedAndRetainedFilesSet.toString()));
+    Sets.SetView<String> retainedAndRemovedFilesSet = Sets.intersection(retainedFilesSet, removedFilesSet);
+    Preconditions.checkState(retainedAndRemovedFilesSet.isEmpty(),
+        String.format("Files present in both retained and removed sets: %s", retainedAndRemovedFilesSet.toString()));
+
+    // validate that a subDir is not present in multiple lists
+    Set<String> addedSubDirsSet = subDirsAdded.stream().map(DirDiff::getDirName).collect(Collectors.toSet());
+    Set<String> retainedSubDirsSet = subDirsRetained.stream().map(DirDiff::getDirName).collect(Collectors.toSet());
+    Set<String> removedSubDirsSet = subDirsRemoved.stream().map(DirIndex::getDirName).collect(Collectors.toSet());
+    Sets.SetView<String> addedAndRetainedSubDirsSet = Sets.intersection(addedSubDirsSet, retainedSubDirsSet);
+    Preconditions.checkState(addedAndRetainedSubDirsSet.isEmpty(),
+        String.format("Sub-dirs present in both added and retained sets: %s", addedAndRetainedSubDirsSet.toString()));
+    Sets.SetView<String> retainedAndRemovedSubDirsSet = Sets.intersection(retainedSubDirsSet, removedSubDirsSet);
+    Preconditions.checkState(retainedAndRemovedSubDirsSet.isEmpty(),
+        String.format("Sub-dirs present in both retained and removed sets: %s", retainedAndRemovedSubDirsSet.toString()));
+
+    this.dirName = dirName;
+    this.filesAdded = filesAdded;
+    this.filesRetained = filesRetained;
+    this.filesRemoved = filesRemoved;
+    this.subDirsAdded = subDirsAdded;
+    this.subDirsRetained = subDirsRetained;
+    this.subDirsRemoved = subDirsRemoved;
+  }
+
+  public String getDirName() {
+    return dirName;
+  }
+
+  public List<File> getFilesAdded() {
+    return filesAdded;
+  }
+
+  public List<FileIndex> getFilesRetained() {
+    return filesRetained;
+  }
+
+  public List<FileIndex> getFilesRemoved() {
+    return filesRemoved;
+  }
+
+  public List<DirDiff> getSubDirsAdded() {
+    return subDirsAdded;
+  }
+
+  public List<DirDiff> getSubDirsRetained() {
+    return subDirsRetained;
+  }
+
+  public List<DirIndex> getSubDirsRemoved() {
+    return subDirsRemoved;
+  }
+
+  public static Stats getStats(DirDiff dirDiff) {
+    Stats stats = new Stats();
+    updateStats(dirDiff, stats);
+    return stats;
+  }
+
+  private static void updateStats(DirDiff dirDiff, Stats stats) {
+    stats.filesAdded += dirDiff.getFilesAdded().size();
+    stats.filesRetained += dirDiff.getFilesRetained().size();
+    stats.filesRemoved += dirDiff.getFilesRemoved().size();
+
+    stats.bytesAdded += dirDiff.getFilesAdded().stream().mapToLong(File::length).sum();
+    stats.bytesRetained += dirDiff.getFilesRetained().stream().mapToLong(f -> f.getFileMetadata().getSize()).sum();
+    stats.bytesRemoved += dirDiff.getFilesRemoved().stream().mapToLong(f -> f.getFileMetadata().getSize()).sum();
+
+    for (DirDiff subDirAdded: dirDiff.getSubDirsAdded()) {
+      stats.subDirsAdded += 1;
+      updateStats(subDirAdded, stats);
+    }
+    for (DirDiff subDirRetained: dirDiff.getSubDirsRetained()) {
+      stats.subDirsRetained += 1;
+      updateStats(subDirRetained, stats);
+    }
+    for (DirIndex subDirRemoved: dirDiff.getSubDirsRemoved()) {
+      stats.subDirsRemoved += 1;
+      updateStatsForDirRemoved(subDirRemoved, stats);
+    }
+  }
+
+  private static void updateStatsForDirRemoved(DirIndex dirIndex, Stats stats) {
+    // every file and sub-dir present in a removed parent dir are to be removed as well
+    // files and sub-dirs to be removed don't matter since they would have already been
+    // cleaned up after the previous commit
+    stats.filesRemoved += dirIndex.getFilesRemoved().size();
+    stats.bytesRemoved += dirIndex.getFilesPresent().stream().mapToLong(f -> f.getFileMetadata().getSize()).sum();
+    for (DirIndex subDirRemoved: dirIndex.getSubDirsPresent()) {
+      stats.subDirsRemoved += 1;
+      updateStatsForDirRemoved(subDirRemoved, stats);
+    }
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+
+    if (o == null || getClass() != o.getClass()) return false;
+
+    DirDiff dirDiff = (DirDiff) o;
+
+    return new EqualsBuilder()
+        .append(getDirName(), dirDiff.getDirName())
+        .append(getFilesAdded(), dirDiff.getFilesAdded())
+        .append(getFilesRetained(), dirDiff.getFilesRetained())
+        .append(getFilesRemoved(), dirDiff.getFilesRemoved())
+        .append(getSubDirsAdded(), dirDiff.getSubDirsAdded())
+        .append(getSubDirsRetained(), dirDiff.getSubDirsRetained())
+        .append(getSubDirsRemoved(), dirDiff.getSubDirsRemoved())
+        .isEquals();
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder(17, 37)
+        .append(getDirName())
+        .append(getFilesAdded())
+        .append(getFilesRetained())
+        .append(getFilesRemoved())
+        .append(getSubDirsAdded())
+        .append(getSubDirsRetained())
+        .append(getSubDirsRemoved())
+        .toHashCode();
+  }
+
+  public static class Stats {
+    public int filesAdded;
+    public int filesRetained;
+    public int filesRemoved;
+
+    public int subDirsAdded;
+    public int subDirsRetained;
+    public int subDirsRemoved;
+
+    public long bytesAdded;
+    public long bytesRetained;
+    public long bytesRemoved;
+
+    @Override
+    public String toString() {
+      return "Stats{" +
+          "filesAdded=" + filesAdded +
+          ", filesRetained=" + filesRetained +
+          ", filesRemoved=" + filesRemoved +
+          ", subDirsAdded=" + subDirsAdded +
+          ", subDirsRetained=" + subDirsRetained +
+          ", subDirsRemoved=" + subDirsRemoved +
+          ", bytesAdded=" + bytesAdded +
+          ", bytesRetained=" + bytesRetained +
+          ", bytesRemoved=" + bytesRemoved +
+          '}';
+    }
+  }
+}
diff --git a/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/DirIndex.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/DirIndex.java
new file mode 100644
index 0000000..31819be
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/DirIndex.java
@@ -0,0 +1,176 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.storage.blobstore.index;
+
+import com.google.common.base.Preconditions;
+import java.util.List;
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+
+/**
+ * Representation of a directory in the blob store
+ */
+public class DirIndex {
+  public static final String ROOT_DIR_NAME = "";
+  private static final short SCHEMA_VERSION = 1;
+
+  private final String dirName;
+
+  private final List<FileIndex> filesPresent;
+  private final List<FileIndex> filesRemoved;
+
+  // Note: subDirsPresent can also have filesRemoved and subDirsRemoved within them.
+  private final List<DirIndex> subDirsPresent;
+  private final List<DirIndex> subDirsRemoved;
+
+  public DirIndex(String dirName,
+      List<FileIndex> filesPresent, List<FileIndex> filesRemoved,
+      List<DirIndex> subDirsPresent, List<DirIndex> subDirsRemoved) {
+    Preconditions.checkNotNull(dirName); // may be empty for root dirs
+    Preconditions.checkNotNull(filesPresent);
+    Preconditions.checkNotNull(filesRemoved);
+    Preconditions.checkNotNull(subDirsPresent);
+    Preconditions.checkNotNull(subDirsRemoved);
+    this.dirName = dirName;
+    this.filesPresent = filesPresent;
+    this.filesRemoved = filesRemoved;
+    this.subDirsPresent = subDirsPresent;
+    this.subDirsRemoved = subDirsRemoved;
+  }
+
+  public static short getSchemaVersion() {
+    return SCHEMA_VERSION;
+  }
+
+  public String getDirName() {
+    return dirName;
+  }
+
+  public List<FileIndex> getFilesPresent() {
+    return filesPresent;
+  }
+
+  public List<FileIndex> getFilesRemoved() {
+    return filesRemoved;
+  }
+
+  public List<DirIndex> getSubDirsPresent() {
+    return subDirsPresent;
+  }
+
+  public List<DirIndex> getSubDirsRemoved() {
+    return subDirsRemoved;
+  }
+
+  public static Stats getStats(DirIndex dirIndex) {
+    Stats stats = new Stats();
+    updateStats(dirIndex, stats);
+    return stats;
+  }
+
+  private static void updateStats(DirIndex dirIndex, Stats stats) {
+    stats.filesPresent += dirIndex.getFilesPresent().size();
+    stats.filesRemoved += dirIndex.getFilesRemoved().size();
+
+    stats.subDirsPresent += dirIndex.getSubDirsPresent().size();
+    stats.subDirsRemoved += dirIndex.getSubDirsRemoved().size();
+
+    stats.bytesPresent += dirIndex.getFilesPresent().stream().mapToLong(fi -> fi.getFileMetadata().getSize()).sum();
+    stats.bytesRemoved += dirIndex.getFilesRemoved().stream().mapToLong(fi -> fi.getFileMetadata().getSize()).sum();
+
+    for (DirIndex subDirPresent : dirIndex.getSubDirsPresent()) {
+      updateStats(subDirPresent, stats);
+    }
+
+    for (DirIndex subDirsRemoved : dirIndex.getSubDirsRemoved()) {
+      updateStatsForSubDirsRemoved(subDirsRemoved, stats);
+    }
+  }
+
+  private static void updateStatsForSubDirsRemoved(DirIndex dirIndex, Stats stats) {
+    stats.filesRemoved += dirIndex.getFilesPresent().size();
+    stats.bytesRemoved += dirIndex.getFilesPresent().stream().mapToLong(fi -> fi.getFileMetadata().getSize()).sum();
+    for (DirIndex subDirToRemove : dirIndex.getSubDirsPresent()) {
+      updateStatsForSubDirsRemoved(subDirToRemove, stats);
+    }
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+
+    if (o == null || getClass() != o.getClass()) return false;
+
+    DirIndex that = (DirIndex) o;
+
+    return new EqualsBuilder()
+        .append(getDirName(), that.getDirName())
+        .append(getFilesPresent(), that.getFilesPresent())
+        .append(getFilesRemoved(), that.getFilesRemoved())
+        .append(getSubDirsPresent(), that.getSubDirsPresent())
+        .append(getSubDirsRemoved(), that.getSubDirsRemoved())
+        .isEquals();
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder(17, 37)
+        .append(getDirName()).append(getFilesPresent())
+        .append(getFilesRemoved())
+        .append(getSubDirsPresent())
+        .append(getSubDirsRemoved())
+        .toHashCode();
+  }
+
+  @Override
+  public String toString() {
+    return "DirIndex{" +
+        "dirName='" +
+        dirName + '\'' +
+        ", filesPresent=" + filesPresent +
+        ", filesRemoved=" + filesRemoved +
+        ", subDirsPresent=" + subDirsPresent +
+        ", subDirsRemoved=" + subDirsRemoved +
+        '}';
+  }
+
+  public static class Stats {
+    public int filesPresent;
+    public int filesRemoved;
+
+    public int subDirsPresent;
+    public int subDirsRemoved;
+
+    public long bytesPresent;
+    public long bytesRemoved;
+
+    @Override
+    public String toString() {
+      return "Stats{" +
+          "filesAdded=" + filesPresent +
+          ", filesRemoved=" + filesRemoved +
+          ", subDirsAdded=" + subDirsPresent +
+          ", subDirsRemoved=" + subDirsRemoved +
+          ", bytesAdded=" + bytesPresent +
+          ", bytesRemoved=" + bytesRemoved +
+          '}';
+    }
+  }
+}
diff --git a/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/FileBlob.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/FileBlob.java
new file mode 100644
index 0000000..8a289aa
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/FileBlob.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.storage.blobstore.index;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+
+
+/**
+ * Representation of a File in a Blob store
+ */
+public class FileBlob {
+
+  private final String blobId;
+  /**
+   * Offset of this blob in the file. A file can be uploaded multiple chunks, and can have
+   * multiple blobs associated with it. Each blob then has its own ID and an offset in the file.
+   */
+  private final int offset;
+
+  public FileBlob(String blobId, int offset) {
+    Preconditions.checkState(StringUtils.isNotBlank(blobId));
+    Preconditions.checkState(offset >= 0);
+    this.blobId = blobId;
+    this.offset = offset;
+  }
+
+  public String getBlobId() {
+    return blobId;
+  }
+
+  public int getOffset() {
+    return offset;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+
+    if (!(o instanceof FileBlob)) {
+      return false;
+    }
+
+    FileBlob fileBlob = (FileBlob) o;
+
+    return new EqualsBuilder()
+        .append(blobId, fileBlob.blobId)
+        .append(offset, fileBlob.offset)
+        .isEquals();
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder(17, 37)
+        .append(blobId)
+        .append(offset)
+        .toHashCode();
+  }
+
+  @Override
+  public String toString() {
+    return "FileBlob{" + "blobId='" + blobId + '\'' + ", offset=" + offset + '}';
+  }
+}
diff --git a/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/FileIndex.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/FileIndex.java
new file mode 100644
index 0000000..b0666cb
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/FileIndex.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.storage.blobstore.index;
+
+import com.google.common.base.Preconditions;
+import java.util.List;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+
+
+/**
+ * Representation of a file in blob store
+ */
+public class FileIndex {
+  private final String fileName;
+  /**
+   * Chunks of file uploaded to blob store as {@link FileBlob}s
+   */
+  private final List<FileBlob> fileBlobs;
+  /**
+   * Metadata (e.g. POSIX file attributes) associated with the file.
+   */
+  private final FileMetadata fileMetadata;
+  /**
+   * Checksum of the file for verifying integrity.
+   */
+  private final long checksum;
+
+
+  public FileIndex(String fileName, List<FileBlob> fileBlobs, FileMetadata fileMetadata, long checksum) {
+    Preconditions.checkState(StringUtils.isNotBlank(fileName));
+    Preconditions.checkNotNull(fileBlobs);
+    // fileBlobs can be empty list for a file of size 0 bytes.
+    Preconditions.checkNotNull(fileMetadata);
+    this.fileName = fileName;
+    this.fileBlobs = fileBlobs;
+    this.fileMetadata = fileMetadata;
+    this.checksum = checksum;
+  }
+
+  public String getFileName() {
+    return fileName;
+  }
+
+  public List<FileBlob> getBlobs() {
+    return fileBlobs;
+  }
+
+  public FileMetadata getFileMetadata() {
+    return fileMetadata;
+  }
+
+  public long getChecksum() {
+    return checksum;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+
+    if (!(o instanceof FileIndex)) {
+      return false;
+    }
+
+    FileIndex that = (FileIndex) o;
+
+    return new EqualsBuilder()
+        .append(getFileName(), that.getFileName())
+        .append(getBlobs(), that.getBlobs())
+        .append(fileMetadata, that.fileMetadata)
+        .append(getChecksum(), that.getChecksum())
+        .isEquals();
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder(17, 37)
+        .append(getFileName())
+        .append(getBlobs())
+        .append(fileMetadata)
+        .append(getChecksum())
+        .toHashCode();
+  }
+
+  @Override
+  public String toString() {
+    return "FileIndex{" +
+        "fileName='" + fileName + '\'' +
+        ", fileBlobs=" + fileBlobs +
+        ", fileMetadata=" + fileMetadata +
+        ", checksum='" + checksum + '\'' +
+        '}';
+  }
+}
\ No newline at end of file
diff --git a/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/FileMetadata.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/FileMetadata.java
new file mode 100644
index 0000000..41db126
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/FileMetadata.java
@@ -0,0 +1,138 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.storage.blobstore.index;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.attribute.PosixFileAttributes;
+import java.nio.file.attribute.PosixFilePermissions;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+
+
+/**
+ * Representation of metadata associated with a File.
+ */
+public class FileMetadata {
+
+  private final long creationTimeMillis;
+  private final long lastModifiedTimeMillis;
+  private final long size;
+  private final String owner;
+  private final String group;
+  private final String permissions;
+
+  public FileMetadata(long creationTimeMillis, long lastModifiedTimeMillis, long size,
+      String owner, String group, String permissions) {
+    Preconditions.checkState(creationTimeMillis >= 0);
+    Preconditions.checkState(lastModifiedTimeMillis >= 0);
+    Preconditions.checkState(size >= 0);
+    Preconditions.checkState(StringUtils.isNotBlank(owner));
+    Preconditions.checkState(StringUtils.isNotBlank(group));
+    Preconditions.checkState(StringUtils.isNotBlank(permissions));
+    this.creationTimeMillis = creationTimeMillis;
+    this.lastModifiedTimeMillis = lastModifiedTimeMillis;
+    this.size = size;
+    this.owner = owner;
+    this.group = group;
+    this.permissions = permissions;
+  }
+
+  public static FileMetadata fromFile(File file) throws IOException {
+    PosixFileAttributes attributes = Files.readAttributes(file.toPath(), PosixFileAttributes.class);
+
+    return new FileMetadata(attributes.creationTime().toMillis(), attributes.lastModifiedTime().toMillis(),
+        attributes.size(), attributes.owner().toString(), attributes.group().toString(),
+        PosixFilePermissions.toString(attributes.permissions()));
+  }
+
+  public long getCreationTimeMillis() {
+    return creationTimeMillis;
+  }
+
+  public long getLastModifiedTimeMillis() {
+    return lastModifiedTimeMillis;
+  }
+
+  public long getSize() {
+    return size;
+  }
+
+  public String getOwner() {
+    return owner;
+  }
+
+  public String getGroup() {
+    return group;
+  }
+
+  public String getPermissions() {
+    return permissions;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+
+    if (!(o instanceof FileMetadata)) {
+      return false;
+    }
+
+    FileMetadata that = (FileMetadata) o;
+
+    return new EqualsBuilder()
+        .append(getCreationTimeMillis(), that.getCreationTimeMillis())
+        .append(getLastModifiedTimeMillis(), that.getLastModifiedTimeMillis())
+        .append(getSize(), that.getSize())
+        .append(getOwner(), that.getOwner())
+        .append(getGroup(), that.getGroup())
+        .append(getPermissions(), that.getPermissions())
+        .isEquals();
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder(17, 37)
+        .append(getCreationTimeMillis())
+        .append(getLastModifiedTimeMillis())
+        .append(getSize())
+        .append(getOwner())
+        .append(getGroup())
+        .append(getPermissions())
+        .toHashCode();
+  }
+
+  @Override
+  public String toString() {
+    return "FileMetadata{" +
+        "creationTimeMillis=" + creationTimeMillis +
+        ", lastModifiedTimeMillis=" + lastModifiedTimeMillis +
+        ", size=" + size +
+        ", owner='" + owner + '\'' +
+        ", group='" + group + '\'' +
+        ", permissions=" + permissions +
+        '}';
+  }
+}
diff --git a/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/SnapshotIndex.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/SnapshotIndex.java
new file mode 100644
index 0000000..b3f68dc
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/SnapshotIndex.java
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.storage.blobstore.index;
+
+import com.google.common.base.Preconditions;
+import java.util.Optional;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+
+
+/**
+ * A {@link SnapshotIndex} contains all the information necessary for recreating the local store by
+ * downloading its contents from the remote blob store. The {@link SnapshotIndex} is itself serialized
+ * and stored as a blob in the remote store, and its blob id tracked in the Task checkpoint.
+ */
+public class SnapshotIndex {
+  private static final short SCHEMA_VERSION = 1;
+
+  private final long creationTimeMillis;
+  /**
+   * Metadata for a snapshot like job name, job Id, store name etc.
+   */
+  private final SnapshotMetadata snapshotMetadata;
+  private final DirIndex dirIndex;
+
+  /**
+   * Blob ID of previous snapshot index blob. Tracked here to be cleaned up
+   * in cleanup phase of commit lifecycle.
+   */
+  private final Optional<String> prevSnapshotIndexBlobId;
+
+  public SnapshotIndex(long creationTimeMillis, SnapshotMetadata snapshotMetadata, DirIndex dirIndex,
+      Optional<String> prevSnapshotIndexBlobId) {
+    Preconditions.checkState(creationTimeMillis >= 0);
+    Preconditions.checkNotNull(snapshotMetadata);
+    Preconditions.checkNotNull(dirIndex);
+    Preconditions.checkNotNull(prevSnapshotIndexBlobId);
+    Preconditions.checkState(
+        !(prevSnapshotIndexBlobId.isPresent() && StringUtils.isBlank(prevSnapshotIndexBlobId.get())));
+    this.creationTimeMillis = creationTimeMillis;
+    this.snapshotMetadata = snapshotMetadata;
+    this.dirIndex = dirIndex;
+    // if no previous snapshot index blob was present, this can be null
+    this.prevSnapshotIndexBlobId = prevSnapshotIndexBlobId;
+  }
+
+  public static short getSchemaVersion() {
+    return SCHEMA_VERSION;
+  }
+
+  public long getCreationTimeMillis() {
+    return creationTimeMillis;
+  }
+
+  public SnapshotMetadata getSnapshotMetadata() {
+    return snapshotMetadata;
+  }
+
+  public DirIndex getDirIndex() {
+    return dirIndex;
+  }
+
+  public Optional<String> getPrevSnapshotIndexBlobId() {
+    return prevSnapshotIndexBlobId;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+
+    if (!(o instanceof SnapshotIndex)) {
+      return false;
+    }
+
+    SnapshotIndex that = (SnapshotIndex) o;
+
+    return new EqualsBuilder()
+        .append(getCreationTimeMillis(), that.getCreationTimeMillis())
+        .append(getSnapshotMetadata(), that.getSnapshotMetadata())
+        .append(getDirIndex(), that.getDirIndex())
+        .append(getPrevSnapshotIndexBlobId(), that.getPrevSnapshotIndexBlobId())
+        .isEquals();
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder(17, 37)
+        .append(getCreationTimeMillis())
+        .append(getSnapshotMetadata())
+        .append(getDirIndex())
+        .append(prevSnapshotIndexBlobId)
+        .toHashCode();
+  }
+
+  @Override
+  public String toString() {
+    return "SnapshotIndex{" +
+        "creationTimeMillis=" + creationTimeMillis +
+        ", snapshotMetadata=" + snapshotMetadata +
+        ", dirIndex=" + dirIndex +
+        ", prevSnapshotIndexBlobId" + prevSnapshotIndexBlobId +
+        '}';
+  }
+}
\ No newline at end of file
diff --git a/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/SnapshotMetadata.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/SnapshotMetadata.java
new file mode 100644
index 0000000..7f059b9
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/SnapshotMetadata.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.storage.blobstore.index;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.samza.checkpoint.CheckpointId;
+
+
+/**
+ * Represents metadata associated with a remote snapshot.
+ */
+public class SnapshotMetadata {
+  private final CheckpointId checkpointId;
+  private final String jobName;
+  private final String jobId;
+  private final String taskName;
+  private final String storeName;
+
+  public SnapshotMetadata(CheckpointId checkpointId, String jobName, String jobId, String taskName,
+      String storeName) {
+    Preconditions.checkNotNull(checkpointId);
+    Preconditions.checkState(StringUtils.isNotBlank(jobName));
+    Preconditions.checkState(StringUtils.isNotBlank(jobId));
+    Preconditions.checkState(StringUtils.isNotBlank(taskName));
+    Preconditions.checkState(StringUtils.isNotBlank(storeName));
+    this.checkpointId = checkpointId;
+    this.jobName = jobName;
+    this.jobId = jobId;
+    this.taskName = taskName;
+    this.storeName = storeName;
+  }
+
+  public CheckpointId getCheckpointId() {
+    return checkpointId;
+  }
+
+  public String getJobName() {
+    return jobName;
+  }
+
+  public String getJobId() {
+    return jobId;
+  }
+
+  public String getTaskName() {
+    return taskName;
+  }
+
+  public String getStoreName() {
+    return storeName;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+
+    if (!(o instanceof SnapshotMetadata)) {
+      return false;
+    }
+
+    SnapshotMetadata that = (SnapshotMetadata) o;
+
+    return new EqualsBuilder()
+        .append(getCheckpointId(), that.getCheckpointId())
+        .append(getJobName(), that.getJobName())
+        .append(getJobId(), that.getJobId())
+        .append(getTaskName(), that.getTaskName())
+        .append(getStoreName(), that.getStoreName())
+        .isEquals();
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder(17, 37)
+        .append(getCheckpointId())
+        .append(getJobName())
+        .append(getJobId())
+        .append(getTaskName())
+        .append(getStoreName())
+        .toHashCode();
+  }
+
+  @Override
+  public String toString() {
+    return "SnapshotMetadata{" +
+        "checkpointId=" + checkpointId +
+        ", jobName='" + jobName + '\'' +
+        ", jobId='" + jobId + '\'' +
+        ", taskName='" + taskName + '\'' +
+        ", storeName='" + storeName + '\'' +
+        '}';
+  }
+}
diff --git a/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/JsonDirIndexMixin.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/JsonDirIndexMixin.java
new file mode 100644
index 0000000..7a29c83
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/JsonDirIndexMixin.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.storage.blobstore.index.serde;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import java.util.List;
+import org.apache.samza.storage.blobstore.index.DirIndex;
+import org.apache.samza.storage.blobstore.index.FileIndex;
+
+
+/**
+ * A mix-in Jackson class to convert {@link DirIndex} to/from JSON.
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
+public abstract class JsonDirIndexMixin {
+
+  @JsonCreator
+  public JsonDirIndexMixin(@JsonProperty("dir-name") String dirName,
+      @JsonProperty("files-present") List<FileIndex> filesPresent,
+      @JsonProperty("files-removed") List<FileIndex> filesRemoved,
+      @JsonProperty("sub-dirs-present") List<DirIndex> subDirsPresent,
+      @JsonProperty("sub-dirs-removed") List<DirIndex> subDirsRemoved) {
+  }
+
+  @JsonProperty("dir-name")
+  abstract String getDirName();
+
+  @JsonProperty("files-present")
+  abstract List<FileIndex> getFilesPresent();
+
+  @JsonProperty("files-removed")
+  abstract List<FileIndex> getFilesRemoved();
+
+  @JsonProperty("sub-dirs-present")
+  abstract List<DirIndex> getSubDirsPresent();
+
+  @JsonProperty("sub-dirs-removed")
+  abstract List<DirIndex> getSubDirsRemoved();
+}
diff --git a/samza-api/src/main/java/org/apache/samza/storage/TaskStorageAdmin.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/JsonFileBlobMixin.java
similarity index 57%
rename from samza-api/src/main/java/org/apache/samza/storage/TaskStorageAdmin.java
rename to samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/JsonFileBlobMixin.java
index 205077b..03a91d1 100644
--- a/samza-api/src/main/java/org/apache/samza/storage/TaskStorageAdmin.java
+++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/JsonFileBlobMixin.java
@@ -17,14 +17,25 @@
  * under the License.
  */
 
-package org.apache.samza.storage;
+package org.apache.samza.storage.blobstore.index.serde;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
 
 /**
- * Creates and validate resources for the StateBackendFactory
+ * A mix-in Jackson class to convert FileBlob to/from JSON.
  */
-public interface TaskStorageAdmin {
+@JsonIgnoreProperties(ignoreUnknown = true)
+public abstract class JsonFileBlobMixin {
+  @JsonCreator
+  public JsonFileBlobMixin(@JsonProperty("blob-id") String blobId, @JsonProperty("offset") int offset) {
+  }
 
-  void createResources();
+  @JsonProperty("blob-id")
+  abstract String getBlobId();
 
-  void validateResources();
+  @JsonProperty("offset")
+  abstract int getOffset();
 }
diff --git a/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/JsonFileIndexMixin.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/JsonFileIndexMixin.java
new file mode 100644
index 0000000..d8ee10a
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/JsonFileIndexMixin.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.storage.blobstore.index.serde;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import java.util.List;
+import org.apache.samza.storage.blobstore.index.FileBlob;
+import org.apache.samza.storage.blobstore.index.FileMetadata;
+
+
+/**
+ * A mix-in Jackson class to convert FileIndex to/from JSON.
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
+public abstract class JsonFileIndexMixin {
+  @JsonCreator
+  public JsonFileIndexMixin(@JsonProperty("file-name") String fileName,
+      @JsonProperty("blobs") List<FileBlob> blobs, @JsonProperty("file-metadata") FileMetadata fileMetadata,
+      @JsonProperty("checksum") long checksum) {
+
+  }
+
+  @JsonProperty("file-name")
+  abstract String getFileName();
+
+  @JsonProperty("blobs")
+  abstract List<FileBlob> getBlobs();
+
+  @JsonProperty("file-metadata")
+  abstract FileMetadata getFileMetadata();
+
+  @JsonProperty("checksum")
+  abstract long getChecksum();
+}
diff --git a/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/JsonFileMetadataMixin.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/JsonFileMetadataMixin.java
new file mode 100644
index 0000000..71eea61
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/JsonFileMetadataMixin.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.storage.blobstore.index.serde;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * A mix-in Jackson class to convert FileMetadata to/from JSON.
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
+public abstract class JsonFileMetadataMixin {
+  @JsonCreator
+  public JsonFileMetadataMixin(@JsonProperty("creation-time-millis") long creationTimeMillis,
+      @JsonProperty("last-modified-time-millis") long lastModifiedTimeMillis, @JsonProperty("size") long size,
+      @JsonProperty("owner") String owner, @JsonProperty("group") String group,
+      @JsonProperty("permissions") String permissions) {
+  }
+
+  @JsonProperty("creation-time-millis")
+  abstract long getCreationTimeMillis();
+
+  @JsonProperty("last-modified-time-millis")
+  abstract long getLastModifiedTimeMillis();
+
+  @JsonProperty("size")
+  abstract long getSize();
+
+  @JsonProperty("owner")
+  abstract String getOwner();
+
+  @JsonProperty("group")
+  abstract String getGroup();
+
+  @JsonProperty("permissions")
+  abstract String getPermissions();
+}
diff --git a/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/JsonSnapshotIndexMixin.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/JsonSnapshotIndexMixin.java
new file mode 100644
index 0000000..3cce253
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/JsonSnapshotIndexMixin.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.storage.blobstore.index.serde;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import java.util.Optional;
+import org.apache.samza.storage.blobstore.index.DirIndex;
+import org.apache.samza.storage.blobstore.index.SnapshotMetadata;
+
+
+/**
+ * A mix-in Jackson class to convert SnapshotIndex to/from JSON.
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
+public abstract class JsonSnapshotIndexMixin {
+  @JsonCreator
+  public JsonSnapshotIndexMixin(@JsonProperty("creation-time-millis") long creationTimeMillis,
+      @JsonProperty("snapshot-metadata") SnapshotMetadata snapshotMetadata,
+      @JsonProperty("dir-index") DirIndex dirIndex,
+      @JsonProperty("prev-snapshot-index-blob-id") Optional<String> prevSnapshotIndexBlobId) {
+  }
+
+  @JsonProperty("creation-time-millis")
+  abstract long getCreationTimeMillis();
+
+  @JsonProperty("snapshot-metadata")
+  abstract SnapshotMetadata getSnapshotMetadata();
+
+  @JsonProperty("dir-index")
+  abstract DirIndex getDirIndex();
+
+  @JsonProperty("prev-snapshot-index-blob-id")
+  abstract Optional<String> getPrevSnapshotIndexBlobId();
+}
diff --git a/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/JsonSnapshotMetadataMixin.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/JsonSnapshotMetadataMixin.java
new file mode 100644
index 0000000..804e330
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/JsonSnapshotMetadataMixin.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.storage.blobstore.index.serde;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.samza.checkpoint.CheckpointId;
+
+/**
+ * A mix-in Jackson class to convert SnapshotMetadata to/from JSON.
+ */
+@JsonIgnoreProperties(ignoreUnknown = true)
+public abstract class JsonSnapshotMetadataMixin {
+  @JsonCreator
+  public JsonSnapshotMetadataMixin(@JsonProperty("checkpoint-id") CheckpointId checkpointId,
+      @JsonProperty("job-name") String jobName, @JsonProperty("job-id") String jobId,
+      @JsonProperty("task-name") String taskName, @JsonProperty("store-name") String storeName) {
+
+  }
+
+  @JsonProperty("checkpoint-id")
+  abstract CheckpointId getCheckpointId();
+
+  @JsonProperty("job-name")
+  abstract String getJobName();
+
+  @JsonProperty("job-id")
+  abstract String getJobId();
+
+  @JsonProperty("task-name")
+  abstract String getTaskName();
+
+  @JsonProperty("store-name")
+  abstract String getStoreName();
+}
diff --git a/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/SnapshotIndexSerde.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/SnapshotIndexSerde.java
new file mode 100644
index 0000000..2e0e50c
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/index/serde/SnapshotIndexSerde.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.storage.blobstore.index.serde;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.ObjectWriter;
+import com.fasterxml.jackson.datatype.jdk8.Jdk8Module;
+import org.apache.samza.SamzaException;
+import org.apache.samza.checkpoint.CheckpointId;
+import org.apache.samza.serializers.JsonCheckpointIdMixin;
+import org.apache.samza.serializers.Serde;
+import org.apache.samza.storage.blobstore.index.DirIndex;
+import org.apache.samza.storage.blobstore.index.FileBlob;
+import org.apache.samza.storage.blobstore.index.FileIndex;
+import org.apache.samza.storage.blobstore.index.FileMetadata;
+import org.apache.samza.storage.blobstore.index.SnapshotIndex;
+import org.apache.samza.storage.blobstore.index.SnapshotMetadata;
+
+
+public class SnapshotIndexSerde implements Serde<SnapshotIndex> {
+
+  private final static ObjectMapper MAPPER = new ObjectMapper();
+  private TypeReference<SnapshotIndex> typeReference;
+  private final ObjectWriter objectWriter;
+
+  public SnapshotIndexSerde() {
+    MAPPER.registerModule(new Jdk8Module());
+    MAPPER.addMixIn(SnapshotIndex.class, JsonSnapshotIndexMixin.class)
+        .addMixIn(SnapshotMetadata.class, JsonSnapshotMetadataMixin.class)
+        .addMixIn(DirIndex.class, JsonDirIndexMixin.class)
+        .addMixIn(FileIndex.class, JsonFileIndexMixin.class)
+        .addMixIn(FileMetadata.class, JsonFileMetadataMixin.class)
+        .addMixIn(FileBlob.class, JsonFileBlobMixin.class)
+        .addMixIn(CheckpointId.class, JsonCheckpointIdMixin.class);
+
+    this.typeReference = new TypeReference<SnapshotIndex>() { };
+    this.objectWriter = MAPPER.writerFor(typeReference);
+  }
+
+  @Override
+  public SnapshotIndex fromBytes(byte[] bytes) {
+    try {
+      return MAPPER.readerFor(typeReference).readValue(bytes);
+    } catch (Exception exception) {
+      throw new SamzaException(String.format("Exception in deserializing SnapshotIndex bytes %s",
+          new String(bytes)), exception);
+    }
+  }
+
+  @Override
+  public byte[] toBytes(SnapshotIndex snapshotIndex) {
+    try {
+      return objectWriter.writeValueAsBytes(snapshotIndex);
+    } catch (Exception exception) {
+      throw new SamzaException(String.format("Exception in serializing SnapshotIndex bytes %s", snapshotIndex), exception);
+    }
+  }
+}
diff --git a/samza-core/src/main/java/org/apache/samza/storage/blobstore/metrics/BlobStoreBackupManagerMetrics.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/metrics/BlobStoreBackupManagerMetrics.java
new file mode 100644
index 0000000..5e41073
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/metrics/BlobStoreBackupManagerMetrics.java
@@ -0,0 +1,136 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.storage.blobstore.metrics;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.samza.metrics.Counter;
+import org.apache.samza.metrics.Gauge;
+import org.apache.samza.metrics.MetricsRegistry;
+import org.apache.samza.metrics.Timer;
+
+
+public class BlobStoreBackupManagerMetrics {
+  private static final String GROUP = BlobStoreBackupManagerMetrics.class.getName();
+  private final MetricsRegistry metricsRegistry;
+
+  // TODO LOW shesharm  per-task throughput
+  public final Gauge<Long> initNs;
+
+  public final Timer uploadNs;
+  // gauges of AtomicLong so that the value can be incremented/decremented atomically in a thread-safe way.
+  // don't set the gauge value directly. use gauge.getValue().incrementAndGet() etc instead.
+  public final Gauge<AtomicLong> filesToUpload;
+  public final Gauge<AtomicLong> bytesToUpload;
+  public final Gauge<AtomicLong> filesUploaded;
+  public final Gauge<AtomicLong> bytesUploaded;
+  public final Gauge<AtomicLong> filesRemaining;
+  public final Gauge<AtomicLong> bytesRemaining;
+  public final Gauge<AtomicLong> filesToRetain;
+  public final Gauge<AtomicLong> bytesToRetain;
+  public final Counter uploadRate;
+
+  // per store breakdowns
+  public final Map<String, Timer> storeDirDiffNs;
+  public final Map<String, Timer> storeUploadNs;
+
+  public final Map<String, Gauge<Long>> storeFilesToUpload;
+  public final Map<String, Gauge<Long>> storeFilesToRetain;
+  public final Map<String, Gauge<Long>> storeFilesToRemove;
+  public final Map<String, Gauge<Long>> storeSubDirsToUpload;
+  public final Map<String, Gauge<Long>> storeSubDirsToRetain;
+  public final Map<String, Gauge<Long>> storeSubDirsToRemove;
+  public final Map<String, Gauge<Long>> storeBytesToUpload;
+  public final Map<String, Gauge<Long>> storeBytesToRetain;
+  public final Map<String, Gauge<Long>> storeBytesToRemove;
+
+  public final Timer cleanupNs;
+
+  // TODO shesharm LOW move to SamzaHistogram
+  public final Timer avgFileUploadNs; // avg time for each file uploaded
+  public final Timer avgFileSizeBytes; // avg size of each file uploaded
+
+  public BlobStoreBackupManagerMetrics(MetricsRegistry metricsRegistry) {
+    this.metricsRegistry = metricsRegistry;
+
+    this.initNs = metricsRegistry.newGauge(GROUP, "init-ns", 0L);
+
+    this.uploadNs = metricsRegistry.newTimer(GROUP, "upload-ns");
+
+    this.filesToUpload = metricsRegistry.newGauge(GROUP, "files-to-upload", new AtomicLong(0L));
+    this.bytesToUpload = metricsRegistry.newGauge(GROUP, "bytes-to-upload", new AtomicLong(0L));
+    this.filesUploaded = metricsRegistry.newGauge(GROUP, "files-uploaded", new AtomicLong(0L));
+    this.bytesUploaded = metricsRegistry.newGauge(GROUP, "bytes-uploaded", new AtomicLong(0L));
+    this.filesRemaining = metricsRegistry.newGauge(GROUP, "files-remaining", new AtomicLong(0L));
+    this.bytesRemaining = metricsRegistry.newGauge(GROUP, "bytes-remaining", new AtomicLong(0L));
+    this.filesToRetain = metricsRegistry.newGauge(GROUP, "files-to-retain", new AtomicLong(0L));
+    this.bytesToRetain = metricsRegistry.newGauge(GROUP, "bytes-to-retain", new AtomicLong(0L));
+
+    this.storeDirDiffNs = new ConcurrentHashMap<>();
+    this.storeUploadNs = new ConcurrentHashMap<>();
+
+    this.storeFilesToUpload = new ConcurrentHashMap<>();
+    this.storeFilesToRetain = new ConcurrentHashMap<>();
+    this.storeFilesToRemove = new ConcurrentHashMap<>();
+    this.storeSubDirsToUpload = new ConcurrentHashMap<>();
+    this.storeSubDirsToRetain = new ConcurrentHashMap<>();
+    this.storeSubDirsToRemove = new ConcurrentHashMap<>();
+    this.storeBytesToUpload = new ConcurrentHashMap<>();
+    this.storeBytesToRetain = new ConcurrentHashMap<>();
+    this.storeBytesToRemove = new ConcurrentHashMap<>();
+
+    this.uploadRate = metricsRegistry.newCounter(GROUP, "upload-rate");
+
+    this.cleanupNs = metricsRegistry.newTimer(GROUP, "cleanup-ns");
+
+    this.avgFileUploadNs = metricsRegistry.newTimer(GROUP,  "avg-file-upload-ns");
+    this.avgFileSizeBytes = metricsRegistry.newTimer(GROUP, "avg-file-size-bytes");
+  }
+
+  public void initStoreMetrics(Collection<String> storeNames) {
+    for (String storeName: storeNames) {
+      storeDirDiffNs.putIfAbsent(storeName,
+          metricsRegistry.newTimer(GROUP, String.format("%s-dir-diff-ns", storeName)));
+      storeUploadNs.putIfAbsent(storeName,
+          metricsRegistry.newTimer(GROUP, String.format("%s-upload-ns", storeName)));
+
+      storeFilesToUpload.putIfAbsent(storeName,
+          metricsRegistry.newGauge(GROUP, String.format("%s-files-to-upload", storeName), 0L));
+      storeFilesToRetain.putIfAbsent(storeName,
+          metricsRegistry.newGauge(GROUP, String.format("%s-files-to-retain", storeName), 0L));
+      storeFilesToRemove.putIfAbsent(storeName,
+          metricsRegistry.newGauge(GROUP, String.format("%s-files-to-remove", storeName), 0L));
+      storeSubDirsToUpload.putIfAbsent(storeName,
+          metricsRegistry.newGauge(GROUP, String.format("%s-sub-dirs-to-upload", storeName), 0L));
+      storeSubDirsToRetain.putIfAbsent(storeName,
+          metricsRegistry.newGauge(GROUP, String.format("%s-sub-dirs-to-retain", storeName), 0L));
+      storeSubDirsToRemove.putIfAbsent(storeName,
+          metricsRegistry.newGauge(GROUP, String.format("%s-sub-dirs-to-remove", storeName), 0L));
+      storeBytesToUpload.putIfAbsent(storeName,
+          metricsRegistry.newGauge(GROUP, String.format("%s-bytes-to-upload", storeName), 0L));
+      storeBytesToRetain.putIfAbsent(storeName,
+          metricsRegistry.newGauge(GROUP, String.format("%s-bytes-to-retain", storeName), 0L));
+      storeBytesToRemove.putIfAbsent(storeName,
+          metricsRegistry.newGauge(GROUP, String.format("%s-bytes-to-remove", storeName), 0L));
+    }
+  }
+}
diff --git a/samza-core/src/main/java/org/apache/samza/storage/blobstore/metrics/BlobStoreRestoreManagerMetrics.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/metrics/BlobStoreRestoreManagerMetrics.java
new file mode 100644
index 0000000..ac07f60
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/metrics/BlobStoreRestoreManagerMetrics.java
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.storage.blobstore.metrics;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.samza.metrics.Counter;
+import org.apache.samza.metrics.Gauge;
+import org.apache.samza.metrics.MetricsRegistry;
+import org.apache.samza.metrics.Timer;
+
+
+public class BlobStoreRestoreManagerMetrics {
+  private static final String GROUP = BlobStoreRestoreManagerMetrics.class.getName();
+  private final MetricsRegistry metricsRegistry;
+
+  // TODO LOW shesharma per-task throughput
+  public final Gauge<Long> initNs;
+  public final Gauge<Long> getSnapshotIndexNs;
+
+  public final Gauge<Long> restoreNs;
+  // gauges of AtomicLong so that the value can be incremented/decremented atomically in a thread-safe way.
+  // don't set the gauge value directly. use gauge.getValue().incrementAndGet() etc instead.
+  public final Gauge<AtomicLong> filesToRestore;
+  public final Gauge<AtomicLong> bytesToRestore;
+  public final Gauge<AtomicLong> filesRestored;
+  public final Gauge<AtomicLong> bytesRestored;
+  public final Gauge<AtomicLong> filesRemaining;
+  public final Gauge<AtomicLong> bytesRemaining;
+
+  public final Counter restoreRate;
+
+  // per store breakdowns
+  public final Map<String, Gauge<Long>> storePreRestoreNs;
+  public final Map<String, Gauge<Long>> storeRestoreNs;
+  public final Map<String, Gauge<Long>> storePostRestoreNs;
+
+  // TODO LOW shesharma move to SamzaHistogram
+  public final Timer avgFileRestoreNs; // avg time for each file restored
+
+  public BlobStoreRestoreManagerMetrics(MetricsRegistry metricsRegistry) {
+    this.metricsRegistry = metricsRegistry;
+
+    this.initNs = metricsRegistry.newGauge(GROUP, "init-ns", 0L);
+    this.getSnapshotIndexNs = metricsRegistry.newGauge(GROUP, "get-snapshot-index-ns", 0L);
+
+    this.restoreNs = metricsRegistry.newGauge(GROUP, "restore-ns", 0L);
+    this.filesToRestore = metricsRegistry.newGauge(GROUP, "files-to-restore", new AtomicLong());
+    this.bytesToRestore = metricsRegistry.newGauge(GROUP, "bytes-to-restore", new AtomicLong());
+    this.filesRestored = metricsRegistry.newGauge(GROUP, "files-restored", new AtomicLong());
+    this.bytesRestored = metricsRegistry.newGauge(GROUP, "bytes-restored", new AtomicLong());
+    this.filesRemaining = metricsRegistry.newGauge(GROUP, "files-remaining", new AtomicLong());
+    this.bytesRemaining = metricsRegistry.newGauge(GROUP, "bytes-remaining", new AtomicLong());
+
+
+    this.restoreRate = metricsRegistry.newCounter(GROUP, "restore-rate");
+
+    this.storePreRestoreNs = new ConcurrentHashMap<>();
+    this.storeRestoreNs = new ConcurrentHashMap<>();
+    this.storePostRestoreNs = new ConcurrentHashMap<>();
+
+    this.avgFileRestoreNs = metricsRegistry.newTimer(GROUP, "avg-file-restore-ns");
+  }
+
+  public void initStoreMetrics(Collection<String> storeNames) {
+    for (String storeName: storeNames) {
+      storePreRestoreNs.putIfAbsent(storeName,
+          metricsRegistry.newGauge(GROUP, String.format("%s-pre-restore-ns", storeName), 0L));
+      storeRestoreNs.putIfAbsent(storeName,
+          metricsRegistry.newGauge(GROUP, String.format("%s-restore-ns", storeName), 0L));
+      storePostRestoreNs.putIfAbsent(storeName,
+          metricsRegistry.newGauge(GROUP, String.format("%s-post-restore-ns", storeName), 0L));
+    }
+  }
+}
diff --git a/samza-core/src/main/java/org/apache/samza/storage/blobstore/util/BlobStoreUtil.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/util/BlobStoreUtil.java
new file mode 100644
index 0000000..9c8b61f
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/util/BlobStoreUtil.java
@@ -0,0 +1,622 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.storage.blobstore.util;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableMap;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.CompletionStage;
+import java.util.concurrent.ExecutorService;
+import java.util.function.Predicate;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import java.util.zip.CRC32;
+import java.util.zip.CheckedInputStream;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.samza.SamzaException;
+import org.apache.samza.checkpoint.Checkpoint;
+import org.apache.samza.checkpoint.CheckpointV2;
+import org.apache.samza.storage.blobstore.BlobStoreManager;
+import org.apache.samza.storage.blobstore.BlobStoreStateBackendFactory;
+import org.apache.samza.storage.blobstore.Metadata;
+import org.apache.samza.storage.blobstore.diff.DirDiff;
+import org.apache.samza.storage.blobstore.exceptions.DeletedException;
+import org.apache.samza.storage.blobstore.exceptions.RetriableException;
+import org.apache.samza.storage.blobstore.index.DirIndex;
+import org.apache.samza.storage.blobstore.index.FileBlob;
+import org.apache.samza.storage.blobstore.index.FileIndex;
+import org.apache.samza.storage.blobstore.index.FileMetadata;
+import org.apache.samza.storage.blobstore.index.SnapshotIndex;
+import org.apache.samza.storage.blobstore.index.SnapshotMetadata;
+import org.apache.samza.storage.blobstore.index.serde.SnapshotIndexSerde;
+import org.apache.samza.storage.blobstore.metrics.BlobStoreBackupManagerMetrics;
+import org.apache.samza.storage.blobstore.metrics.BlobStoreRestoreManagerMetrics;
+import org.apache.samza.util.FutureUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Helper methods to interact with remote blob store service and GET/PUT/DELETE a
+ * {@link SnapshotIndex} or {@link DirDiff}.
+ */
+public class BlobStoreUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(BlobStoreUtil.class);
+
+  private final BlobStoreManager blobStoreManager;
+  private final ExecutorService executor;
+  private final BlobStoreBackupManagerMetrics backupMetrics;
+  private final BlobStoreRestoreManagerMetrics restoreMetrics;
+
+  public BlobStoreUtil(BlobStoreManager blobStoreManager, ExecutorService executor,
+      BlobStoreBackupManagerMetrics backupMetrics, BlobStoreRestoreManagerMetrics restoreMetrics) {
+    this.blobStoreManager = blobStoreManager;
+    this.executor = executor;
+    this.backupMetrics = backupMetrics;
+    this.restoreMetrics = restoreMetrics;
+  }
+
+  /**
+   * Get the blob id of {@link SnapshotIndex} and {@link SnapshotIndex}es for the provided {@code task}
+   * in the provided {@code checkpoint}.
+   * @param jobName job name is used to build request metadata
+   * @param jobId job id is used to build request metadata
+   * @param taskName task name to get the store state checkpoint markers and snapshot indexes for
+   * @param checkpoint {@link Checkpoint} instance to get the store state checkpoint markers from. Only
+   *                   {@link CheckpointV2} and newer are supported for blob stores.
+   * @return Map of store name to its blob id of snapshot indices and their corresponding snapshot indices for the task.
+   */
+  public Map<String, Pair<String, SnapshotIndex>> getStoreSnapshotIndexes(
+      String jobName, String jobId, String taskName, Checkpoint checkpoint) {
+    //TODO MED shesharma document error handling (checkpoint ver, blob not found, getBlob)
+    if (checkpoint == null) {
+      LOG.debug("No previous checkpoint found for taskName: {}", taskName);
+      return ImmutableMap.of();
+    }
+
+    if (checkpoint.getVersion() == 1) {
+      throw new SamzaException("Checkpoint version 1 is not supported for blob store backup and restore.");
+    }
+
+    Map<String, CompletableFuture<Pair<String, SnapshotIndex>>>
+        storeSnapshotIndexFutures = new HashMap<>();
+
+    CheckpointV2 checkpointV2 = (CheckpointV2) checkpoint;
+    Map<String, Map<String, String>> factoryToStoreSCMs = checkpointV2.getStateCheckpointMarkers();
+    Map<String, String> storeSnapshotIndexBlobIds = factoryToStoreSCMs.get(BlobStoreStateBackendFactory.class.getName());
+
+    if (storeSnapshotIndexBlobIds != null) {
+      storeSnapshotIndexBlobIds.forEach((storeName, snapshotIndexBlobId) -> {
+        try {
+          LOG.debug("Getting snapshot index for taskName: {} store: {} blobId: {}", taskName, storeName, snapshotIndexBlobId);
+          Metadata requestMetadata =
+              new Metadata(Metadata.SNAPSHOT_INDEX_PAYLOAD_PATH, Optional.empty(), jobName, jobId, taskName, storeName);
+          CompletableFuture<SnapshotIndex> snapshotIndexFuture =
+              getSnapshotIndex(snapshotIndexBlobId, requestMetadata).toCompletableFuture();
+          Pair<CompletableFuture<String>, CompletableFuture<SnapshotIndex>> pairOfFutures =
+              Pair.of(CompletableFuture.completedFuture(snapshotIndexBlobId), snapshotIndexFuture);
+
+          // save the future and block once in the end instead of blocking for each request.
+          storeSnapshotIndexFutures.put(storeName, FutureUtil.toFutureOfPair(pairOfFutures));
+        } catch (Exception e) {
+          throw new SamzaException(
+              String.format("Error getting SnapshotIndex for blobId: %s for taskName: %s store: %s",
+                  snapshotIndexBlobId, taskName, storeName), e);
+        }
+      });
+    } else {
+      LOG.debug("No store SCMs found for blob store state backend in for taskName: {} in checkpoint {}",
+          taskName, checkpointV2.getCheckpointId());
+    }
+
+    try {
+      return FutureUtil.toFutureOfMap(t -> {
+        Throwable unwrappedException = FutureUtil.unwrapExceptions(CompletionException.class, t);
+        if (unwrappedException instanceof DeletedException) {
+          LOG.warn("Ignoring already deleted snapshot index for taskName: {}", taskName, t);
+          return true;
+        } else {
+          return false;
+        }
+      }, storeSnapshotIndexFutures).join();
+    } catch (Exception e) {
+      throw new SamzaException(
+          String.format("Error while waiting to get store snapshot indexes for task %s", taskName), e);
+    }
+  }
+
+  /**
+   * GETs the {@link SnapshotIndex} from the blob store.
+   * @param blobId blob ID of the {@link SnapshotIndex} to get
+   * @return a Future containing the {@link SnapshotIndex}
+   */
+  public CompletableFuture<SnapshotIndex> getSnapshotIndex(String blobId, Metadata metadata) {
+    Preconditions.checkState(StringUtils.isNotBlank(blobId));
+    String opName = "getSnapshotIndex: " + blobId;
+    return FutureUtil.executeAsyncWithRetries(opName, () -> {
+      ByteArrayOutputStream indexBlobStream = new ByteArrayOutputStream(); // no need to close ByteArrayOutputStream
+      return blobStoreManager.get(blobId, indexBlobStream, metadata).toCompletableFuture()
+          .thenApplyAsync(f -> new SnapshotIndexSerde().fromBytes(indexBlobStream.toByteArray()), executor);
+    }, isCauseNonRetriable(), executor);
+  }
+
+  /**
+   * PUTs the {@link SnapshotIndex} to the blob store.
+   * @param snapshotIndex SnapshotIndex to put.
+   * @return a Future containing the blob ID of the {@link SnapshotIndex}.
+   */
+  public CompletableFuture<String> putSnapshotIndex(SnapshotIndex snapshotIndex) {
+    byte[] bytes = new SnapshotIndexSerde().toBytes(snapshotIndex);
+    String opName = "putSnapshotIndex for checkpointId: " + snapshotIndex.getSnapshotMetadata().getCheckpointId();
+    return FutureUtil.executeAsyncWithRetries(opName, () -> {
+      InputStream inputStream = new ByteArrayInputStream(bytes); // no need to close ByteArrayInputStream
+      SnapshotMetadata snapshotMetadata = snapshotIndex.getSnapshotMetadata();
+      Metadata metadata = new Metadata(Metadata.SNAPSHOT_INDEX_PAYLOAD_PATH, Optional.of((long) bytes.length),
+          snapshotMetadata.getJobName(), snapshotMetadata.getJobId(), snapshotMetadata.getTaskName(),
+          snapshotMetadata.getStoreName());
+      return blobStoreManager.put(inputStream, metadata).toCompletableFuture();
+    }, isCauseNonRetriable(), executor);
+  }
+
+  /**
+   * WARNING: This method deletes the **SnapshotIndex blob** from the snapshot. This should only be called to clean
+   * up an older snapshot **AFTER** all the files and sub-dirs to be deleted from this snapshot are already deleted
+   * using {@link #cleanUpDir(DirIndex, Metadata)}
+   *
+   * @param snapshotIndexBlobId blob ID of SnapshotIndex blob to delete
+   * @return a future that completes when the index blob is deleted from remote store.
+   */
+  public CompletionStage<Void> deleteSnapshotIndexBlob(String snapshotIndexBlobId, Metadata metadata) {
+    Preconditions.checkState(StringUtils.isNotBlank(snapshotIndexBlobId));
+    LOG.debug("Deleting SnapshotIndex blob: {} from blob store", snapshotIndexBlobId);
+    String opName = "deleteSnapshotIndexBlob: " + snapshotIndexBlobId;
+    return FutureUtil.executeAsyncWithRetries(opName, () ->
+        blobStoreManager.delete(snapshotIndexBlobId, metadata).toCompletableFuture(), isCauseNonRetriable(), executor);
+  }
+
+  /**
+   * Non-blocking restore of a {@link SnapshotIndex} to local store by downloading all the files and sub-dirs associated
+   * with this remote snapshot.
+   * @return A future that completes when all the async downloads completes
+   */
+  public CompletableFuture<Void> restoreDir(File baseDir, DirIndex dirIndex, Metadata metadata) {
+    LOG.debug("Restoring contents of directory: {} from remote snapshot.", baseDir);
+
+    List<CompletableFuture<Void>> downloadFutures = new ArrayList<>();
+
+    try {
+      // create parent directories if they don't exist
+      Files.createDirectories(baseDir.toPath());
+    } catch (IOException exception) {
+      LOG.error("Error creating directory: {} for restore", baseDir.getAbsolutePath(), exception);
+      throw new SamzaException(String.format("Error creating directory: %s for restore",
+          baseDir.getAbsolutePath()), exception);
+    }
+
+    // restore all files in the directory
+    for (FileIndex fileIndex : dirIndex.getFilesPresent()) {
+      File fileToRestore = Paths.get(baseDir.getAbsolutePath(), fileIndex.getFileName()).toFile();
+      Metadata requestMetadata =
+          new Metadata(fileToRestore.getAbsolutePath(), Optional.of(fileIndex.getFileMetadata().getSize()),
+              metadata.getJobName(), metadata.getJobId(), metadata.getTaskName(), metadata.getStoreName());
+      List<FileBlob> fileBlobs = fileIndex.getBlobs();
+
+      String opName = "restoreFile: " + fileToRestore.getAbsolutePath();
+      CompletableFuture<Void> fileRestoreFuture =
+          FutureUtil.executeAsyncWithRetries(opName, () -> getFile(fileBlobs, fileToRestore, requestMetadata),
+              isCauseNonRetriable(), executor);
+      downloadFutures.add(fileRestoreFuture);
+    }
+
+    // restore any sub-directories
+    List<DirIndex> subDirs = dirIndex.getSubDirsPresent();
+    for (DirIndex subDir : subDirs) {
+      File subDirFile = Paths.get(baseDir.getAbsolutePath(), subDir.getDirName()).toFile();
+      downloadFutures.add(restoreDir(subDirFile, subDir, metadata));
+    }
+
+    return FutureUtil.allOf(downloadFutures);
+  }
+
+  /**
+   * Recursively upload all new files and upload or update contents of all subdirs in the {@link DirDiff} and return a
+   * Future containing the {@link DirIndex} associated with the directory.
+   * @param dirDiff diff for the contents of this directory
+   * @return A future with the {@link DirIndex} if the upload completed successfully.
+   */
+  public CompletionStage<DirIndex> putDir(DirDiff dirDiff, SnapshotMetadata snapshotMetadata) {
+    // Upload all new files in the dir
+    List<File> filesToUpload = dirDiff.getFilesAdded();
+    List<CompletionStage<FileIndex>> fileFutures = filesToUpload.stream()
+        .map(file -> putFile(file, snapshotMetadata))
+        .collect(Collectors.toList());
+
+    CompletableFuture<Void> allFilesFuture =
+        CompletableFuture.allOf(fileFutures.toArray(new CompletableFuture[0]));
+
+    List<CompletionStage<DirIndex>> subDirFutures = new ArrayList<>();
+    // recursively upload all new subdirs of this dir
+    for (DirDiff subDirAdded: dirDiff.getSubDirsAdded()) {
+      subDirFutures.add(putDir(subDirAdded, snapshotMetadata));
+    }
+    // recursively update contents of all subdirs that are retained but might have been modified
+    for (DirDiff subDirRetained: dirDiff.getSubDirsRetained()) {
+      subDirFutures.add(putDir(subDirRetained, snapshotMetadata));
+    }
+    CompletableFuture<Void> allDirBlobsFuture =
+        CompletableFuture.allOf(subDirFutures.toArray(new CompletableFuture[0]));
+
+    return CompletableFuture.allOf(allDirBlobsFuture, allFilesFuture)
+        .thenApplyAsync(f -> {
+          LOG.trace("All file and dir uploads complete for task: {} store: {}",
+              snapshotMetadata.getTaskName(), snapshotMetadata.getStoreName());
+          List<FileIndex> filesPresent = fileFutures.stream()
+              .map(blob -> blob.toCompletableFuture().join())
+              .collect(Collectors.toList());
+
+          filesPresent.addAll(dirDiff.getFilesRetained());
+
+          List<DirIndex> subDirsPresent = subDirFutures.stream()
+              .map(subDir -> subDir.toCompletableFuture().join())
+              .collect(Collectors.toList());
+
+          LOG.debug("Uploaded diff for task: {} store: {} with statistics: {}",
+              snapshotMetadata.getTaskName(), snapshotMetadata.getStoreName(), DirDiff.getStats(dirDiff));
+
+          LOG.trace("Returning new DirIndex for task: {} store: {}",
+              snapshotMetadata.getTaskName(), snapshotMetadata.getStoreName());
+          return new DirIndex(dirDiff.getDirName(),
+              filesPresent,
+              dirDiff.getFilesRemoved(),
+              subDirsPresent,
+              dirDiff.getSubDirsRemoved());
+        }, executor);
+  }
+
+  /**
+   * WARNING: Recursively delete **ALL** the associated files and subdirs within the provided {@link DirIndex}.
+   * @param dirIndex {@link DirIndex} whose entire contents are to be deleted.
+   * @param metadata {@link Metadata} related to the request
+   * @return a future that completes when ALL the files and subdirs associated with the dirIndex have been
+   * marked for deleted in the remote blob store.
+   */
+  public CompletionStage<Void> deleteDir(DirIndex dirIndex, Metadata metadata) {
+    LOG.debug("Completely deleting dir: {} in blob store", dirIndex.getDirName());
+    List<CompletionStage<Void>> deleteFutures = new ArrayList<>();
+    // Delete all files present in subDir
+    for (FileIndex file: dirIndex.getFilesPresent()) {
+      Metadata requestMetadata =
+          new Metadata(file.getFileName(), Optional.of(file.getFileMetadata().getSize()),
+              metadata.getJobName(), metadata.getJobId(), metadata.getTaskName(), metadata.getStoreName());
+      deleteFutures.add(deleteFile(file, requestMetadata));
+    }
+
+    // Delete all subDirs present recursively
+    for (DirIndex subDir: dirIndex.getSubDirsPresent()) {
+      deleteFutures.add(deleteDir(subDir, metadata));
+    }
+
+    return CompletableFuture.allOf(deleteFutures.toArray(new CompletableFuture[0]));
+  }
+
+
+  /**
+   * Recursively issue delete requests for files and dirs marked to be removed in a previously created remote snapshot.
+   * Note: We do not immediately delete files/dirs to be removed when uploading a snapshot to the remote
+   * store. We just track them for deletion during the upload, and delete them AFTER the snapshot is uploaded, and the
+   * blob IDs have been persisted as part of the checkpoint. This is to prevent data loss if a failure happens
+   * part way through the commit. We issue delete these file/subdirs in cleanUp() phase of commit lifecycle.
+   * @param dirIndex the dir in the remote snapshot to clean up.
+   * @param metadata Metadata related to the request
+   * @return a future that completes when all the files and subdirs marked for deletion are cleaned up.
+   */
+  public CompletionStage<Void> cleanUpDir(DirIndex dirIndex, Metadata metadata) {
+    String dirName = dirIndex.getDirName();
+    if (DirIndex.ROOT_DIR_NAME.equals(dirName)) {
+      LOG.debug("Cleaning up root dir in blob store.");
+    } else {
+      LOG.debug("Cleaning up dir: {} in blob store.", dirIndex.getDirName());
+    }
+
+    List<CompletionStage<Void>> cleanUpFuture = new ArrayList<>();
+    List<FileIndex> files = dirIndex.getFilesRemoved();
+    for (FileIndex file: files) {
+      Metadata requestMetadata =
+          new Metadata(file.getFileName(), Optional.of(file.getFileMetadata().getSize()), metadata.getJobName(),
+              metadata.getJobId(), metadata.getTaskName(), metadata.getStoreName());
+      cleanUpFuture.add(deleteFile(file, requestMetadata));
+    }
+
+    for (DirIndex subDirToDelete : dirIndex.getSubDirsRemoved()) {
+      // recursively delete ALL contents of the subDirToDelete.
+      cleanUpFuture.add(deleteDir(subDirToDelete, metadata));
+    }
+
+    for (DirIndex subDirToRetain : dirIndex.getSubDirsPresent()) {
+      // recursively clean up the subDir, only deleting files and subdirs marked for deletion.
+      cleanUpFuture.add(cleanUpDir(subDirToRetain, metadata));
+    }
+
+    return CompletableFuture.allOf(cleanUpFuture.toArray(new CompletableFuture[0]));
+  }
+
+  /**
+   * Gets a file from the blob store.
+   * @param fileBlobs List of {@link FileBlob}s that constitute this file.
+   * @param fileToRestore File pointing to the local path where the file will be restored.
+   * @param requestMetadata {@link Metadata} associated with this request
+   * @return a future that completes when the file is downloaded and written or if an exception occurs.
+   */
+  @VisibleForTesting
+  CompletableFuture<Void> getFile(List<FileBlob> fileBlobs, File fileToRestore, Metadata requestMetadata) {
+    FileOutputStream outputStream = null;
+    try {
+      long restoreFileStartTime = System.nanoTime();
+      if (fileToRestore.exists()) {
+        // delete the file if it already exists, e.g. from a previous retry.
+        Files.delete(fileToRestore.toPath());
+      }
+
+      outputStream = new FileOutputStream(fileToRestore);
+      final FileOutputStream finalOutputStream = outputStream;
+      // TODO HIGH shesharm add integration tests to ensure empty files and directories are handled correctly E2E.
+      fileToRestore.createNewFile(); // create file for 0 byte files (fileIndex entry but no fileBlobs).
+      // create a copy to ensure list being sorted is mutable.
+      List<FileBlob> fileBlobsCopy = new ArrayList<>(fileBlobs);
+      fileBlobsCopy.sort(Comparator.comparingInt(FileBlob::getOffset)); // sort by offset.
+
+      // chain the futures such that write to file for blobs is sequential.
+      // can be optimized to write concurrently to the file later.
+      CompletableFuture<Void> resultFuture = CompletableFuture.completedFuture(null);
+      for (FileBlob fileBlob : fileBlobsCopy) {
+        resultFuture = resultFuture.thenComposeAsync(v -> {
+          LOG.debug("Starting restore for file: {} with blob id: {} at offset: {}", fileToRestore, fileBlob.getBlobId(),
+              fileBlob.getOffset());
+          return blobStoreManager.get(fileBlob.getBlobId(), finalOutputStream, requestMetadata);
+        }, executor);
+      }
+
+      resultFuture = resultFuture.thenRunAsync(() -> {
+        LOG.debug("Finished restore for file: {}. Closing output stream.", fileToRestore);
+        try {
+          // flush the file contents to disk
+          finalOutputStream.getFD().sync();
+          finalOutputStream.close();
+        } catch (Exception e) {
+          throw new SamzaException(String.format("Error closing output stream for file: %s", fileToRestore.getAbsolutePath()), e);
+        }
+      }, executor);
+
+      resultFuture.whenComplete((res, ex) -> {
+        if (restoreMetrics != null) {
+          restoreMetrics.avgFileRestoreNs.update(System.nanoTime() - restoreFileStartTime);
+
+          long fileSize = requestMetadata.getPayloadSize();
+          restoreMetrics.restoreRate.inc(fileSize);
+          restoreMetrics.filesRestored.getValue().addAndGet(1);
+          restoreMetrics.bytesRestored.getValue().addAndGet(fileSize);
+          restoreMetrics.filesRemaining.getValue().addAndGet(-1);
+          restoreMetrics.bytesRemaining.getValue().addAndGet(-1 * fileSize);
+        }
+      });
+      return resultFuture;
+    } catch (Exception exception) {
+      try {
+        if (outputStream != null) {
+          outputStream.close();
+        }
+      } catch (Exception err) {
+        LOG.error("Error closing output stream for file: {}", fileToRestore.getAbsolutePath(), err);
+      }
+
+      throw new SamzaException(String.format("Error restoring file: %s in path: %s",
+          fileToRestore.getName(), requestMetadata.getPayloadPath()), exception);
+    }
+  }
+
+  /**
+   * Upload a File to blob store.
+   * @param file File to upload to blob store.
+   * @return A future containing the {@link FileIndex} for the uploaded file.
+   */
+  @VisibleForTesting
+  CompletableFuture<FileIndex> putFile(File file, SnapshotMetadata snapshotMetadata) {
+    if (file == null || !file.isFile()) {
+      String message = file != null ? "Dir or Symbolic link" : "null";
+      throw new SamzaException(String.format("Required a non-null parameter of type file, provided: %s", message));
+    }
+    long putFileStartTime = System.nanoTime();
+
+    String opName = "putFile: " + file.getAbsolutePath();
+    Supplier<CompletionStage<FileIndex>> fileUploadAction = () -> {
+      LOG.debug("Putting file: {} to blob store.", file.getPath());
+      CompletableFuture<FileIndex> fileBlobFuture;
+      CheckedInputStream inputStream = null;
+      try {
+        // TODO HIGH shesharm maybe use the more efficient CRC32C / PureJavaCRC32 impl
+        inputStream = new CheckedInputStream(new FileInputStream(file), new CRC32());
+        CheckedInputStream finalInputStream = inputStream;
+        FileMetadata fileMetadata = FileMetadata.fromFile(file);
+        if (backupMetrics != null) {
+          backupMetrics.avgFileSizeBytes.update(fileMetadata.getSize());
+        }
+
+        Metadata metadata =
+            new Metadata(file.getAbsolutePath(), Optional.of(fileMetadata.getSize()), snapshotMetadata.getJobName(),
+                snapshotMetadata.getJobId(), snapshotMetadata.getTaskName(), snapshotMetadata.getStoreName());
+
+        fileBlobFuture = blobStoreManager.put(inputStream, metadata)
+            .thenApplyAsync(id -> {
+              LOG.trace("Put complete. Closing input stream for file: {}.", file.getPath());
+              try {
+                finalInputStream.close();
+              } catch (Exception e) {
+                throw new SamzaException(String.format("Error closing input stream for file: %s",
+                    file.getAbsolutePath()), e);
+              }
+
+              LOG.trace("Returning new FileIndex for file: {}.", file.getPath());
+              return new FileIndex(
+                  file.getName(),
+                  Collections.singletonList(new FileBlob(id, 0)),
+                  fileMetadata,
+                  finalInputStream.getChecksum().getValue());
+            }, executor).toCompletableFuture();
+      } catch (Exception e) {
+        try {
+          if (inputStream != null) {
+            inputStream.close();
+          }
+        } catch (Exception err) {
+          LOG.error("Error closing input stream for file: {}", file.getName(), err);
+        }
+        LOG.error("Error putting file: {}", file.getName(), e);
+        throw new SamzaException(String.format("Error putting file %s", file.getAbsolutePath()), e);
+      }
+      return fileBlobFuture;
+    };
+
+    return FutureUtil.executeAsyncWithRetries(opName, fileUploadAction, isCauseNonRetriable(), executor)
+        .whenComplete((res, ex) -> {
+          if (backupMetrics != null) {
+            backupMetrics.avgFileUploadNs.update(System.nanoTime() - putFileStartTime);
+
+            long fileSize = file.length();
+            backupMetrics.uploadRate.inc(fileSize);
+            backupMetrics.filesUploaded.getValue().addAndGet(1);
+            backupMetrics.bytesUploaded.getValue().addAndGet(fileSize);
+            backupMetrics.filesRemaining.getValue().addAndGet(-1);
+            backupMetrics.bytesRemaining.getValue().addAndGet(-1 * fileSize);
+          }
+        });
+  }
+
+  /**
+   * Delete a {@link FileIndex} from the remote store by deleting all {@link FileBlob}s associated with it.
+   * @param fileIndex FileIndex of the file to delete from the remote store.
+   * @param metadata
+   * @return a future that completes when the FileIndex has been marked for deletion in the remote blob store.
+   */
+  private CompletionStage<Void> deleteFile(FileIndex fileIndex, Metadata metadata) {
+    List<CompletionStage<Void>> deleteFutures = new ArrayList<>();
+    List<FileBlob> fileBlobs = fileIndex.getBlobs();
+    for (FileBlob fileBlob : fileBlobs) {
+      LOG.debug("Deleting file: {} blobId: {} from blob store.", fileIndex.getFileName(), fileBlob.getBlobId());
+      String opName = "deleteFile: " + fileIndex.getFileName() + " blobId: " + fileBlob.getBlobId();
+      Supplier<CompletionStage<Void>> fileDeletionAction = () ->
+          blobStoreManager.delete(fileBlob.getBlobId(), metadata).toCompletableFuture();
+      CompletableFuture<Void> fileDeletionFuture =
+          FutureUtil.executeAsyncWithRetries(opName, fileDeletionAction, isCauseNonRetriable(), executor);
+      deleteFutures.add(fileDeletionFuture);
+    }
+
+    return CompletableFuture.allOf(deleteFutures.toArray(new CompletableFuture[0]));
+  }
+
+  /**
+   * Recursively mark all the blobs associated with the {@link DirIndex} to never expire (remove TTL).
+   * @param dirIndex the {@link DirIndex} whose contents' TTL needs to be removed
+   * @param metadata {@link Metadata} related to the request
+   * @return A future that completes when all the blobs associated with this dirIndex are marked to
+   * never expire.
+   */
+  private CompletableFuture<Void> removeTTL(DirIndex dirIndex, Metadata metadata) {
+    String dirName = dirIndex.getDirName();
+    if (DirIndex.ROOT_DIR_NAME.equals(dirName)) {
+      LOG.debug("Removing TTL for files and dirs present in DirIndex for root dir.");
+    } else {
+      LOG.debug("Removing TTL for files and dirs present in DirIndex for dir: {}", dirName);
+    }
+
+    List<CompletableFuture<Void>> updateTTLsFuture = new ArrayList<>();
+    for (DirIndex subDir: dirIndex.getSubDirsPresent()) {
+      updateTTLsFuture.add(removeTTL(subDir, metadata));
+    }
+
+    for (FileIndex file: dirIndex.getFilesPresent()) {
+      Metadata requestMetadata =
+          new Metadata(file.getFileName(), Optional.of(file.getFileMetadata().getSize()),
+              metadata.getJobName(), metadata.getJobId(), metadata.getTaskName(), metadata.getStoreName());
+      List<FileBlob> fileBlobs = file.getBlobs();
+      for (FileBlob fileBlob : fileBlobs) {
+        String opname = "removeTTL for fileBlob: " + file.getFileName() + " with blobId: {}" + fileBlob.getBlobId();
+        Supplier<CompletionStage<Void>> ttlRemovalAction = () ->
+            blobStoreManager.removeTTL(fileBlob.getBlobId(), requestMetadata).toCompletableFuture();
+        CompletableFuture<Void> ttlRemovalFuture =
+            FutureUtil.executeAsyncWithRetries(opname, ttlRemovalAction, isCauseNonRetriable(), executor);
+        updateTTLsFuture.add(ttlRemovalFuture);
+      }
+    }
+
+    return CompletableFuture.allOf(updateTTLsFuture.toArray(new CompletableFuture[0]));
+  }
+
+
+  /**
+   * Marks all the blobs associated with an {@link SnapshotIndex} to never expire.
+   * @param snapshotIndex {@link SnapshotIndex} of the remote snapshot
+   * @param metadata {@link Metadata} related to the request
+   * @return A future that completes when all the files and subdirs associated with this remote snapshot are marked to
+   * never expire.
+   */
+  public CompletionStage<Void> removeTTL(String indexBlobId, SnapshotIndex snapshotIndex, Metadata metadata) {
+    SnapshotMetadata snapshotMetadata = snapshotIndex.getSnapshotMetadata();
+    LOG.debug("Marking contents of SnapshotIndex: {} to never expire", snapshotMetadata.toString());
+
+    String opName = "removeTTL for SnapshotIndex for checkpointId: " + snapshotMetadata.getCheckpointId();
+    Supplier<CompletionStage<Void>> removeDirIndexTTLAction =
+      () -> removeTTL(snapshotIndex.getDirIndex(), metadata).toCompletableFuture();
+    CompletableFuture<Void> dirIndexTTLRemovalFuture =
+        FutureUtil.executeAsyncWithRetries(opName, removeDirIndexTTLAction, isCauseNonRetriable(), executor);
+
+    return dirIndexTTLRemovalFuture.thenComposeAsync(aVoid -> {
+      String op2Name = "removeTTL for indexBlobId: " + indexBlobId;
+      Supplier<CompletionStage<Void>> removeIndexBlobTTLAction =
+        () -> blobStoreManager.removeTTL(indexBlobId, metadata).toCompletableFuture();
+      return FutureUtil.executeAsyncWithRetries(op2Name, removeIndexBlobTTLAction, isCauseNonRetriable(), executor);
+    }, executor);
+  }
+
+  private static Predicate<Throwable> isCauseNonRetriable() {
+    return throwable -> {
+      Throwable unwrapped = FutureUtil.unwrapExceptions(CompletionException.class, throwable);
+      return unwrapped != null && !RetriableException.class.isAssignableFrom(unwrapped.getClass());
+    };
+  }
+}
\ No newline at end of file
diff --git a/samza-core/src/main/java/org/apache/samza/storage/blobstore/util/DirDiffUtil.java b/samza-core/src/main/java/org/apache/samza/storage/blobstore/util/DirDiffUtil.java
new file mode 100644
index 0000000..f09b56f
--- /dev/null
+++ b/samza-core/src/main/java/org/apache/samza/storage/blobstore/util/DirDiffUtil.java
@@ -0,0 +1,403 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.storage.blobstore.util;
+
+import com.google.common.base.Preconditions;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.nio.file.attribute.PosixFileAttributes;
+import java.nio.file.attribute.PosixFilePermissions;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.BiPredicate;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.zip.CRC32;
+import java.util.zip.CheckedInputStream;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.samza.SamzaException;
+import org.apache.samza.storage.blobstore.diff.DirDiff;
+import org.apache.samza.storage.blobstore.index.DirIndex;
+import org.apache.samza.storage.blobstore.index.FileIndex;
+import org.apache.samza.storage.blobstore.index.FileMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Provides helper methods to create a {@link DirDiff} between local and remote snapshots.
+ */
+public class DirDiffUtil {
+  private static final Logger LOG = LoggerFactory.getLogger(DirDiffUtil.class);
+
+  /**
+   * Checks if a local directory and a remote directory are identical. Local and remote directories are identical iff:
+   * 1. The local directory has exactly the same set of files as the remote directory, and the files are themselves
+   * identical, as determined by {@link #areSameFile(boolean)}, except for those allowed to differ according to
+   * {@code filesToIgnore}.
+   * 2. The local directory has exactly the same set of sub-directories as the remote directory.
+   *
+   * @param filesToIgnore a set of file names to ignore during the directory comparisons
+   *                      (does not exclude directory names)
+   * @param compareLargeFileChecksums whether to compare checksums for large files (&gt; 1 MB).
+   * @return boolean indicating whether the local and remote directory are identical.
+   */
+  // TODO HIGH shesharm add unit tests
+  public BiPredicate<File, DirIndex> areSameDir(Set<String> filesToIgnore, boolean compareLargeFileChecksums) {
+    return (localDir, remoteDir) -> {
+      String remoteDirName = remoteDir.getDirName().equals(DirIndex.ROOT_DIR_NAME) ? "root" : remoteDir.getDirName();
+      LOG.debug("Creating diff between local dir: {} and remote dir: {} for comparison.",
+          localDir.getAbsolutePath(), remoteDirName);
+      DirDiff dirDiff = DirDiffUtil.getDirDiff(localDir, remoteDir, DirDiffUtil.areSameFile(compareLargeFileChecksums));
+
+      boolean areSameDir = true;
+      List<String> filesRemoved = dirDiff.getFilesRemoved().stream()
+          .map(FileIndex::getFileName)
+          .filter(name -> !filesToIgnore.contains(name))
+          .collect(Collectors.toList());
+
+      if (!filesRemoved.isEmpty()) {
+        areSameDir = false;
+        LOG.error("Local directory: {} is missing files that are present in remote snapshot: {}",
+            localDir.getAbsolutePath(), StringUtils.join(filesRemoved, ", "));
+      }
+
+      List<DirIndex> subDirsRemoved = dirDiff.getSubDirsRemoved();
+      if (!subDirsRemoved.isEmpty()) {
+        areSameDir = false;
+        List<String> missingSubDirs = subDirsRemoved.stream().map(DirIndex::getDirName).collect(Collectors.toList());
+        LOG.error("Local directory: {} is missing sub-dirs that are present in remote snapshot: {}",
+            localDir.getAbsolutePath(), StringUtils.join(missingSubDirs, ", "));
+      }
+
+      List<String> filesAdded = dirDiff.getFilesAdded().stream()
+          .map(File::getName)
+          .filter(name -> !filesToIgnore.contains(name))
+          .collect(Collectors.toList());
+      if (!filesAdded.isEmpty()) {
+        areSameDir = false;
+        LOG.error("Local directory: {} has additional files that are not present in remote snapshot: {}",
+            localDir.getAbsolutePath(), StringUtils.join(filesAdded, ", "));
+      }
+
+      List<DirDiff> subDirsAdded = dirDiff.getSubDirsAdded();
+      if (!subDirsAdded.isEmpty()) {
+        areSameDir = false;
+        List<String> addedDirs = subDirsAdded.stream().map(DirDiff::getDirName).collect(Collectors.toList());
+        LOG.error("Local directory: {} has additional sub-dirs that are not present in remote snapshot: {}",
+            localDir.getAbsolutePath(), StringUtils.join(addedDirs, ", "));
+      }
+
+      // dir diff calculation already ensures that all retained files are equal (by definition)
+      // recursively test that all retained sub-dirs are equal as well
+      Map<String, DirIndex> remoteSubDirs = new HashMap<>();
+      for (DirIndex subDir: remoteDir.getSubDirsPresent()) {
+        remoteSubDirs.put(subDir.getDirName(), subDir);
+      }
+      for (DirDiff subDirRetained: dirDiff.getSubDirsRetained()) {
+        String localSubDirName = subDirRetained.getDirName();
+        File localSubDirFile = Paths.get(localDir.getAbsolutePath(), localSubDirName).toFile();
+        DirIndex remoteSubDir = remoteSubDirs.get(localSubDirName);
+        boolean areSameSubDir = areSameDir(filesToIgnore, false).test(localSubDirFile, remoteSubDir);
+        if (!areSameSubDir) {
+          LOG.debug("Local sub-dir: {} and remote sub-dir: {} are not same.",
+              localSubDirFile.getAbsolutePath(), remoteSubDir.getDirName());
+          areSameDir = false;
+        }
+      }
+
+      LOG.debug("Local dir: {} and remote dir: {} are {}the same.",
+          localDir.getAbsolutePath(), remoteDirName, areSameDir ? "" : "not ");
+      return areSameDir;
+    };
+  }
+
+  /**
+   * Bipredicate to test a local file in the filesystem and a remote file {@link FileIndex} and find out if they represent
+   * the same file. Files with same attributes as well as content are same file. A SST file in a special case. They are
+   * immutable, so we only compare their attributes but not the content.
+   * @param compareLargeFileChecksums whether to compare checksums for large files (&gt; 1 MB).
+   * @return BiPredicate to test similarity of local and remote files
+   */
+  public static BiPredicate<File, FileIndex> areSameFile(boolean compareLargeFileChecksums) {
+    return (localFile, remoteFile) -> {
+      if (localFile.getName().equals(remoteFile.getFileName())) {
+        FileMetadata remoteFileMetadata = remoteFile.getFileMetadata();
+
+        PosixFileAttributes localFileAttrs = null;
+        try {
+          localFileAttrs = Files.readAttributes(localFile.toPath(), PosixFileAttributes.class);
+        } catch (IOException e) {
+          LOG.error("Error reading attributes for file: {}", localFile.getAbsolutePath());
+          throw new RuntimeException(String.format("Error reading attributes for file: %s", localFile.getAbsolutePath()));
+        }
+
+        // Don't compare file timestamps. The ctime of a local file just restored will be different than the
+        // remote file, and will cause the file to be uploaded again during the first commit after restore.
+
+        boolean areSameFiles =
+            localFileAttrs.size() == remoteFileMetadata.getSize() &&
+                localFileAttrs.group().getName().equals(remoteFileMetadata.getGroup()) &&
+                localFileAttrs.owner().getName().equals(remoteFileMetadata.getOwner()) &&
+                PosixFilePermissions.toString(localFileAttrs.permissions()).equals(remoteFileMetadata.getPermissions());
+
+        if (!areSameFiles) {
+          LOG.debug("Local file: {} and remote file: {} are not same. " +
+                  "Local file attributes: {}. Remote file attributes: {}.",
+              localFile.getAbsolutePath(), remoteFile.getFileName(),
+              fileAttributesToString(localFileAttrs), remoteFile.getFileMetadata().toString());
+          return false;
+        } else {
+          LOG.trace("Local file: {}. Remote file: {}. " +
+                  "Local file attributes: {}. Remote file attributes: {}.",
+              localFile.getAbsolutePath(), remoteFile.getFileName(),
+              fileAttributesToString(localFileAttrs), remoteFile.getFileMetadata().toString());
+        }
+
+        boolean isLargeFile = localFileAttrs.size() > 1024 * 1024;
+        if (!compareLargeFileChecksums && isLargeFile) {
+          // Since RocksDB SST files are immutable after creation, we can skip the expensive checksum computations
+          // which requires reading the entire file.
+          LOG.debug("Local file: {} and remote file: {} are same. " +
+                  "Skipping checksum calculation for large file of size: {}.",
+              localFile.getAbsolutePath(), remoteFile.getFileName(), localFileAttrs.size());
+          return true;
+        } else {
+          try {
+            FileInputStream fis = new FileInputStream(localFile);
+            CheckedInputStream cis = new CheckedInputStream(fis, new CRC32());
+            byte[] buffer = new byte[8 * 1024]; // 8 KB
+            while (cis.read(buffer, 0, buffer.length) >= 0) { }
+            long localFileChecksum = cis.getChecksum().getValue();
+            cis.close();
+
+            boolean areSameChecksum = localFileChecksum == remoteFile.getChecksum();
+            if (!areSameChecksum) {
+              LOG.debug("Local file: {} and remote file: {} are not same. " +
+                      "Local checksum: {}. Remote checksum: {}",
+                  localFile.getAbsolutePath(), remoteFile.getFileName(), localFileChecksum, remoteFile.getChecksum());
+            } else {
+              LOG.debug("Local file: {} and remote file: {} are same. Local checksum: {}. Remote checksum: {}",
+                  localFile.getAbsolutePath(), remoteFile.getFileName(), localFileChecksum, remoteFile.getChecksum());
+            }
+            return areSameChecksum;
+          } catch (IOException e) {
+            throw new SamzaException("Error calculating checksum for local file: " + localFile.getAbsolutePath(), e);
+          }
+        }
+      }
+
+      return false;
+    };
+  }
+
+  /**
+   * Compare the local snapshot directory and the remote snapshot directory and return the recursive diff of the two as
+   * a {@link DirDiff}.
+   * @param localSnapshotDir File representing local snapshot root directory
+   * @param remoteSnapshotDir {@link DirIndex} representing the remote snapshot directory
+   * @param areSameFile A BiPredicate to test if a local and remote file are the same file
+   * @return {@link DirDiff} representing the recursive diff of local and remote snapshots directories
+   */
+  public static DirDiff getDirDiff(File localSnapshotDir, DirIndex remoteSnapshotDir,
+      BiPredicate<File, FileIndex> areSameFile) {
+    return getDirDiff(localSnapshotDir, remoteSnapshotDir, areSameFile, true);
+  }
+
+  private static DirDiff getDirDiff(File localSnapshotDir, DirIndex remoteSnapshotDir,
+      BiPredicate<File, FileIndex> areSameFile, boolean isRootDir) {
+    Preconditions.checkState(localSnapshotDir != null && localSnapshotDir.isDirectory());
+    Preconditions.checkNotNull(remoteSnapshotDir);
+
+    LOG.debug("Creating DirDiff between local dir: {} and remote dir: {}",
+        localSnapshotDir.getPath(), remoteSnapshotDir.getDirName());
+    List<DirDiff> subDirsAdded = new ArrayList<>();
+    List<DirDiff> subDirsRetained = new ArrayList<>();
+    List<DirIndex> subDirsRemoved = new ArrayList<>();
+
+    // list files returns empty list if local snapshot directory is empty
+    List<File> localSnapshotFiles = Arrays.asList(Objects.requireNonNull(localSnapshotDir.listFiles(File::isFile)));
+    List<FileIndex> remoteSnapshotFiles = remoteSnapshotDir.getFilesPresent();
+
+    // list files returns empty list if local snapshot directory is empty
+    List<File> localSnapshotSubDirs = Arrays.asList(Objects.requireNonNull(localSnapshotDir.listFiles(File::isDirectory)));
+    Set<String> localSnapshotSubDirNames = localSnapshotSubDirs.stream()
+        .map(File::getName)
+        .collect(Collectors.toCollection(HashSet::new));
+
+    List<DirIndex> remoteSnapshotSubDirs = remoteSnapshotDir.getSubDirsPresent();
+    Set<String> remoteSnapshotSubDirNames = remoteSnapshotSubDirs.stream()
+        .map(DirIndex::getDirName)
+        .collect(Collectors.toCollection(HashSet::new));
+
+    // TODO MED shesharm: this compares each file in directory 3 times. Categorize files in one traversal instead.
+    List<File> filesToUpload = getNewFilesToUpload(remoteSnapshotFiles, localSnapshotFiles, areSameFile);
+    List<FileIndex> filesToRetain = getFilesToRetain(remoteSnapshotFiles, localSnapshotFiles, areSameFile);
+    List<FileIndex> filesToRemove = getFilesToRemove(remoteSnapshotFiles, localSnapshotFiles, areSameFile);
+
+    for (File localSnapshotSubDir: localSnapshotSubDirs) {
+      if (!remoteSnapshotSubDirNames.contains(localSnapshotSubDir.getName())) {
+        LOG.debug("Subdir {} present in local snapshot but not in remote snapshot. " +
+            "Recursively adding subdir contents.", localSnapshotSubDir.getPath());
+        subDirsAdded.add(getDiffForNewDir(localSnapshotSubDir));
+      } else {
+        LOG.debug("Subdir {} present in local snapshot and in remote snapshot. " +
+            "Recursively comparing local and remote subdirs.", localSnapshotSubDir.getPath());
+        DirIndex remoteSubDirIndex =
+            remoteSnapshotSubDirs.stream()
+                .filter(indexBlob -> indexBlob.getDirName().equals(localSnapshotSubDir.getName()))
+                .findFirst().get();
+        subDirsRetained.add(getDirDiff(localSnapshotSubDir, remoteSubDirIndex, areSameFile, false));
+      }
+    }
+
+    // 3. Subdir in remote snapshot but not in local snapshot
+    for (DirIndex remoteSnapshotSubDir: remoteSnapshotSubDirs) {
+      if (!localSnapshotSubDirNames.contains(remoteSnapshotSubDir.getDirName())) {
+        LOG.debug("Subdir {} present in remote snapshot but not in local snapshot. " +
+            "Marking for removal from remote snapshot. ", remoteSnapshotDir.getDirName());
+        subDirsRemoved.add(remoteSnapshotSubDir);
+      }
+    }
+
+    String dirName = isRootDir ? DirIndex.ROOT_DIR_NAME : localSnapshotDir.getName();
+    return new DirDiff(dirName,
+        filesToUpload, filesToRetain, filesToRemove,
+        subDirsAdded, subDirsRetained, subDirsRemoved);
+  }
+
+  /**
+   * Builds a {@link DirDiff} from a new local directory that is not already present in the remote snapshot.
+   * @param localSubDir File representing the local directory to create the new {@link DirDiff} for.
+   */
+  private static DirDiff getDiffForNewDir(File localSubDir) {
+    List<File> filesAdded = new ArrayList<>();
+    List<DirDiff> subDirsAdded = new ArrayList<>();
+
+    File[] files = localSubDir.listFiles();
+    if (files != null) {
+      for (File file: files) {
+        if (file.isFile()) {
+          LOG.debug("Adding file {} to local sub dir {}", file.getName(), localSubDir.getPath());
+          filesAdded.add(file);
+        } else {
+          LOG.debug("Adding sub dir {} to sub dir {}", file.getName(), localSubDir.getPath());
+          subDirsAdded.add(getDiffForNewDir(file));
+        }
+      }
+    }
+
+    return new DirDiff(localSubDir.getName(), filesAdded, Collections.emptyList(), Collections.emptyList(),
+        subDirsAdded, Collections.emptyList(), Collections.emptyList());
+  }
+
+  /**
+   * Returns a list of files uploaded in remote checkpoint that are not present in new local snapshot and needs to be
+   * deleted/reclaimed from remote store.
+   */
+  private static List<FileIndex> getFilesToRemove(
+      List<FileIndex> remoteSnapshotFiles, List<File> localSnapshotFiles,
+      BiPredicate<File, FileIndex> areSameFile) {
+    List<FileIndex> filesToRemove = new ArrayList<>();
+
+    Map<String, File> localFiles = localSnapshotFiles.stream()
+        .collect(Collectors.toMap(File::getName, Function.identity()));
+
+    for (FileIndex remoteFile : remoteSnapshotFiles) {
+      String remoteFileName = remoteFile.getFileName();
+      if (!localFiles.containsKey(remoteFileName) ||
+          !areSameFile.test(localFiles.get(remoteFileName), remoteFile)) {
+        LOG.debug("File {} only present in remote snapshot or is not the same as local file.", remoteFile.getFileName());
+        filesToRemove.add(remoteFile);
+      }
+    }
+
+    return filesToRemove;
+  }
+
+  /**
+   * Returns a list of files to be uploaded to remote store that are part of new snapshot created locally.
+   */
+  private static List<File> getNewFilesToUpload(
+      List<FileIndex> remoteSnapshotFiles, List<File> localSnapshotFiles,
+      BiPredicate<File, FileIndex> areSameFile) {
+    List<File> filesToUpload = new ArrayList<>();
+
+    Map<String, FileIndex> remoteFiles = remoteSnapshotFiles.stream()
+        .collect(Collectors.toMap(FileIndex::getFileName, Function.identity()));
+
+    for (File localFile: localSnapshotFiles) {
+      String localFileName = localFile.getName();
+      if (!remoteFiles.containsKey(localFileName) ||
+          !areSameFile.test(localFile, remoteFiles.get(localFileName))) {
+        LOG.debug("File {} only present in local snapshot or is not the same as remote file.", localFile.getPath());
+        filesToUpload.add(localFile);
+      }
+    }
+
+    return filesToUpload;
+  }
+
+  /**
+   * Returns a list of common files between local and remote snapshot. These files are reused from prev remote snapshot
+   * and do not need to be uploaded again.
+   */
+  private static List<FileIndex> getFilesToRetain(
+      List<FileIndex> remoteSnapshotFiles, List<File> localSnapshotFiles,
+      BiPredicate<File, FileIndex> areSameFile) {
+    List<FileIndex> filesToRetain = new ArrayList<>();
+
+    Map<String, File> localFiles = localSnapshotFiles.stream()
+        .collect(Collectors.toMap(File::getName, Function.identity()));
+
+    for (FileIndex remoteFile : remoteSnapshotFiles) {
+      String remoteFileName = remoteFile.getFileName();
+      if (localFiles.containsKey(remoteFileName) &&
+          areSameFile.test(localFiles.get(remoteFileName), remoteFile)) {
+        String localFilePath = localFiles.get(remoteFileName).getPath();
+        LOG.debug("File {} present in both local and remote snapshot and is the same.", localFilePath);
+        filesToRetain.add(remoteFile);
+      }
+    }
+
+    return filesToRetain;
+  }
+
+  private static String fileAttributesToString(PosixFileAttributes fileAttributes) {
+    return "PosixFileAttributes{" +
+        "creationTimeMillis=" + fileAttributes.creationTime().toMillis() +
+        ", lastModifiedTimeMillis=" + fileAttributes.lastModifiedTime().toMillis() +
+        ", size=" + fileAttributes.size() +
+        ", owner='" + fileAttributes.owner() + '\'' +
+        ", group='" + fileAttributes.group() + '\'' +
+        ", permissions=" + PosixFilePermissions.toString(fileAttributes.permissions()) +
+        '}';
+  }
+}
\ No newline at end of file
diff --git a/samza-core/src/main/java/org/apache/samza/util/FutureUtil.java b/samza-core/src/main/java/org/apache/samza/util/FutureUtil.java
index dc527e9..16e38bc 100644
--- a/samza-core/src/main/java/org/apache/samza/util/FutureUtil.java
+++ b/samza-core/src/main/java/org/apache/samza/util/FutureUtil.java
@@ -146,14 +146,14 @@ public class FutureUtil {
       Supplier<? extends CompletionStage<T>> action,
       Predicate<? extends Throwable> abortRetries,
       ExecutorService executor) {
-    Duration maxDuration = Duration.ofMinutes(1);
+    Duration maxDuration = Duration.ofMinutes(10);
 
     RetryPolicy<Object> retryPolicy = new RetryPolicy<>()
-        .withBackoff(100, 10000, ChronoUnit.MILLIS)
+        .withBackoff(100, 312500, ChronoUnit.MILLIS, 5) // 100 ms, 500 ms, 2500 ms, 12.5 s, 1.05 min, 5.20 min, 5.20 min
         .withMaxDuration(maxDuration)
         .abortOn(abortRetries) // stop retrying if predicate returns true
-        .onRetry(e -> LOG.warn("Action: {} attempt: {} completed with error {} after start. Retrying up to {}.",
-            opName, e.getAttemptCount(), e.getElapsedTime(), maxDuration, e.getLastFailure()));
+        .onRetry(e -> LOG.warn("Action: {} attempt: {} completed with error {} ms after start. Retrying up to {} ms.",
+            opName, e.getAttemptCount(), e.getElapsedTime().toMillis(), maxDuration.toMillis(), e.getLastFailure()));
 
     return Failsafe.with(retryPolicy).with(executor).getStageAsync(action::get);
   }
diff --git a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala
index dceb27b..1e16928 100644
--- a/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala
+++ b/samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala
@@ -25,12 +25,15 @@ import java.net.{URL, UnknownHostException}
 import java.nio.file.Path
 import java.time.Duration
 import java.util
-import java.util.{Base64, Optional}
-import java.util.concurrent.{CountDownLatch, ExecutorService, Executors, ScheduledExecutorService, ThreadPoolExecutor, TimeUnit}
+import java.util.concurrent._
 import java.util.function.Consumer
+import java.util.{Base64, Optional}
+
 import com.google.common.annotations.VisibleForTesting
 import com.google.common.util.concurrent.ThreadFactoryBuilder
+import org.apache.samza.SamzaException
 import org.apache.samza.checkpoint.{CheckpointListener, OffsetManager, OffsetManagerMetrics}
+import org.apache.samza.clustermanager.StandbyTaskUtil
 import org.apache.samza.config.{StreamConfig, _}
 import org.apache.samza.container.disk.DiskSpaceMonitor.Listener
 import org.apache.samza.container.disk.{DiskQuotaPolicyFactory, DiskSpaceMonitor, NoThrottlingDiskQuotaPolicyFactory, PollingScanDiskSpaceMonitor}
@@ -38,7 +41,7 @@ import org.apache.samza.container.host.{StatisticsMonitorImpl, SystemMemoryStati
 import org.apache.samza.context._
 import org.apache.samza.diagnostics.DiagnosticsManager
 import org.apache.samza.job.model.{ContainerModel, JobModel, TaskMode}
-import org.apache.samza.metrics.{JmxServer, JvmMetrics, MetricsRegistry, MetricsRegistryMap, MetricsReporter}
+import org.apache.samza.metrics.{JmxServer, JvmMetrics, MetricsRegistryMap, MetricsReporter}
 import org.apache.samza.serializers._
 import org.apache.samza.serializers.model.SamzaObjectMapper
 import org.apache.samza.startpoint.StartpointManager
@@ -49,8 +52,6 @@ import org.apache.samza.table.TableManager
 import org.apache.samza.task._
 import org.apache.samza.util.ScalaJavaUtil.JavaOptionals
 import org.apache.samza.util.{Util, _}
-import org.apache.samza.SamzaException
-import org.apache.samza.clustermanager.StandbyTaskUtil
 
 import scala.collection.JavaConverters._
 
@@ -345,6 +346,23 @@ object SamzaContainer extends Logging {
 
     info("Got change log system streams: %s" format storeChangelogs)
 
+    /*
+     * This keeps track of the changelog SSPs that are associated with the whole container. This is used so that we can
+     * prefetch the metadata about the all of the changelog SSPs associated with the container whenever we need the
+     * metadata about some of the changelog SSPs.
+     * An example use case is when Samza writes offset files for stores ({@link TaskStorageManager}). Each task is
+     * responsible for its own offset file, but if we can do prefetching, then most tasks will already have cached
+     * metadata by the time they need the offset metadata.
+     * Note: By using all changelog streams to build the sspsToPrefetch, any fetches done for persisted stores will
+     * include the ssps for non-persisted stores, so this is slightly suboptimal. However, this does not increase the
+     * actual number of calls to the {@link SystemAdmin}, and we can decouple this logic from the per-task objects (e.g.
+     * {@link TaskStorageManager}).
+     */
+    val changelogSSPMetadataCache = new SSPMetadataCache(systemAdmins,
+      Duration.ofSeconds(5),
+      SystemClock.instance,
+      getChangelogSSPsForContainer(containerModel, storeChangelogs).asJava)
+
     val intermediateStreams = streamConfig
       .getStreamIds()
       .asScala
@@ -453,6 +471,7 @@ object SamzaContainer extends Logging {
 
     val threadPoolSize = jobConfig.getThreadPoolSize
     info("Got thread pool size: " + threadPoolSize)
+    samzaContainerMetrics.containerThreadPoolSize.set(threadPoolSize)
 
     val taskThreadPool = if (threadPoolSize > 0) {
       Executors.newFixedThreadPool(threadPoolSize,
@@ -500,8 +519,9 @@ object SamzaContainer extends Logging {
     val loggedStorageBaseDir = getLoggedStorageBaseDir(jobConfig, defaultStoreBaseDir)
     info("Got base directory for logged data stores: %s" format loggedStorageBaseDir)
 
+    // TODO dchen should we enforce restore factories to be subset of backup factories?
     val stateStorageBackendRestoreFactory = ReflectionUtil
-      .getObj(storageConfig.getStateBackendRestoreFactory(), classOf[StateBackendFactory])
+      .getObj(storageConfig.getRestoreFactory(), classOf[StateBackendFactory])
 
     val containerStorageManager = new ContainerStorageManager(
       checkpointManager,
@@ -527,7 +547,7 @@ object SamzaContainer extends Logging {
 
     storeWatchPaths.addAll(containerStorageManager.getStoreDirectoryPaths)
 
-    val stateStorageBackendBackupFactories = storageConfig.getStateBackendBackupFactories.asScala.map(
+    val stateStorageBackendBackupFactories = storageConfig.getBackupFactories().asScala.map(
       ReflectionUtil.getObj(_, classOf[StateBackendFactory])
     )
 
@@ -559,7 +579,7 @@ object SamzaContainer extends Logging {
             if (taskInstanceMetrics.contains(taskName) &&
               taskInstanceMetrics.get(taskName).isDefined) taskInstanceMetrics.get(taskName).get.registry
             else new MetricsRegistryMap
-          val taskBackupManager = factory.getBackupManager(jobContext, containerContext,
+          val taskBackupManager = factory.getBackupManager(jobContext, containerModel,
             taskModel, commitThreadPool, taskMetricsRegistry, config, new SystemClock,
             loggedStorageBaseDir, nonLoggedStorageBaseDir)
           taskBackupManagerMap.put(factory.getClass.getName, taskBackupManager)
@@ -616,8 +636,8 @@ object SamzaContainer extends Logging {
 
     val containerMemoryMb : Int = new ClusterManagerConfig(config).getContainerMemoryMb
 
-    val hostStatisticsMonitor : SystemStatisticsMonitor = new StatisticsMonitorImpl()
-    hostStatisticsMonitor.registerListener(new SystemStatisticsMonitor.Listener {
+    val memoryStatisticsMonitor : SystemStatisticsMonitor = new StatisticsMonitorImpl()
+    memoryStatisticsMonitor.registerListener(new SystemStatisticsMonitor.Listener {
       override def onUpdate(sample: SystemMemoryStatistics): Unit = {
         val physicalMemoryBytes : Long = sample.getPhysicalMemoryBytes
         val physicalMemoryMb : Float = physicalMemoryBytes / (1024.0F * 1024.0F)
@@ -625,16 +645,7 @@ object SamzaContainer extends Logging {
         logger.debug("Container physical memory utilization (mb): " + physicalMemoryMb)
         logger.debug("Container physical memory utilization: " + memoryUtilization)
         samzaContainerMetrics.physicalMemoryMb.set(physicalMemoryMb)
-        samzaContainerMetrics.physicalMemoryUtilization.set(memoryUtilization)
-
-        var containerThreadPoolSize : Long = 0
-        var containerActiveThreads : Long = 0
-        if (taskThreadPool != null) {
-          containerThreadPoolSize = taskThreadPool.asInstanceOf[ThreadPoolExecutor].getPoolSize
-          containerActiveThreads = taskThreadPool.asInstanceOf[ThreadPoolExecutor].getActiveCount
-        }
-        samzaContainerMetrics.containerThreadPoolSize.set(containerThreadPoolSize)
-        samzaContainerMetrics.containerActiveThreads.set(containerActiveThreads)
+        samzaContainerMetrics.physicalMemoryUtilization.set(memoryUtilization);
       }
     })
 
@@ -680,7 +691,7 @@ object SamzaContainer extends Logging {
       reporters = reporters,
       jvm = jvm,
       diskSpaceMonitor = diskSpaceMonitor,
-      hostStatisticsMonitor = hostStatisticsMonitor,
+      hostStatisticsMonitor = memoryStatisticsMonitor,
       taskThreadPool = taskThreadPool,
       commitThreadPool = commitThreadPool,
       timerExecutor = timerExecutor,
@@ -690,6 +701,19 @@ object SamzaContainer extends Logging {
       containerStorageManager = containerStorageManager,
       diagnosticsManager = diagnosticsManager)
   }
+
+  /**
+    * Builds the set of SSPs for all changelogs on this container.
+    */
+  @VisibleForTesting
+  private[container] def getChangelogSSPsForContainer(containerModel: ContainerModel,
+    changeLogSystemStreams: util.Map[String, SystemStream]): Set[SystemStreamPartition] = {
+    containerModel.getTasks.values().asScala
+      .map(taskModel => taskModel.getChangelogPartition)
+      .flatMap(changelogPartition => changeLogSystemStreams.asScala.map { case (_, systemStream) =>
+        new SystemStreamPartition(systemStream, changelogPartition) })
+      .toSet
+  }
 }
 
 class SamzaContainer(
@@ -719,7 +743,10 @@ class SamzaContainer(
 
   private val jobConfig = new JobConfig(config)
   private val taskConfig = new TaskConfig(config)
-  val shutdownMs: Long = taskConfig.getShutdownMs
+
+  val shutdownMs: Long = taskConfig.getLong(TaskConfig.TASK_SHUTDOWN_MS, 5000)
+
+  var shutdownHookThread: Thread = null
   var jmxServer: JmxServer = null
 
   @volatile private var status = SamzaContainerStatus.NOT_STARTED
@@ -1132,4 +1159,4 @@ class SamzaContainer(
       hostStatisticsMonitor.stop()
     }
   }
-}
\ No newline at end of file
+}
diff --git a/samza-core/src/main/scala/org/apache/samza/job/local/ProcessJobFactory.scala b/samza-core/src/main/scala/org/apache/samza/job/local/ProcessJobFactory.scala
index 3264932..64e2408 100644
--- a/samza-core/src/main/scala/org/apache/samza/job/local/ProcessJobFactory.scala
+++ b/samza-core/src/main/scala/org/apache/samza/job/local/ProcessJobFactory.scala
@@ -24,7 +24,7 @@ import java.util
 import org.apache.samza.SamzaException
 import org.apache.samza.application.ApplicationUtil
 import org.apache.samza.application.descriptors.ApplicationDescriptorUtil
-import org.apache.samza.config.{Config, JobConfig, TaskConfig}
+import org.apache.samza.config.{BlobStoreConfig, Config, JobConfig, StorageConfig, TaskConfig}
 import org.apache.samza.container.TaskName
 import org.apache.samza.coordinator.metadatastore.{CoordinatorStreamStore, NamespaceAwareCoordinatorStreamStore}
 import org.apache.samza.coordinator.stream.messages.SetChangelogMapping
@@ -34,7 +34,7 @@ import org.apache.samza.job.model.JobModelUtil
 import org.apache.samza.job.{CommandBuilder, ShellCommandBuilder, StreamJob, StreamJobFactory}
 import org.apache.samza.metrics.MetricsRegistryMap
 import org.apache.samza.startpoint.StartpointManager
-import org.apache.samza.storage.ChangelogStreamManager
+import org.apache.samza.storage.{ChangelogStreamManager, StateBackendFactory}
 import org.apache.samza.util.{ConfigUtil, CoordinatorStreamUtil, DiagnosticsUtil, Logging, ReflectionUtil}
 
 import scala.collection.JavaConversions._
@@ -91,6 +91,17 @@ class ProcessJobFactory extends StreamJobFactory with Logging {
     val metadataResourceUtil = new MetadataResourceUtil(jobModel, metricsRegistry, config)
     metadataResourceUtil.createResources()
 
+    val storageConfig = new StorageConfig(config)
+    storageConfig.getBackupFactories.foreach(stateStorageBackendBackupFactory => {
+      val stateBackendFactory : StateBackendFactory =
+        ReflectionUtil.getObj(stateStorageBackendBackupFactory, classOf[StateBackendFactory])
+      val stateBackendAdmin = stateBackendFactory.getAdmin(jobModel, config)
+      // Create resources required for state backend admin
+      stateBackendAdmin.createResources()
+      // Validate resources required for state backend admin
+      stateBackendAdmin.validateResources()
+    })
+
     if (new JobConfig(config).getStartpointEnabled()) {
       // fan out the startpoints
       val startpointManager = new StartpointManager(coordinatorStreamStore)
diff --git a/samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java b/samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java
index 6838ec2..5cc79b3 100644
--- a/samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java
+++ b/samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java
@@ -443,9 +443,9 @@ public class ContainerStorageManager {
       }
 
       for (String storeName : storesToCreate) {
+        List<String> storeBackupManager = storageConfig.getStoreBackupFactory(storeName);
         // A store is considered durable if it is backed by a changelog or another backupManager factory
-        boolean isDurable = changelogSystemStreams.containsKey(storeName) ||
-            !storageConfig.getStoreBackupManagerClassName(storeName).isEmpty();
+        boolean isDurable = changelogSystemStreams.containsKey(storeName) || !storeBackupManager.isEmpty();
         boolean isSideInput = this.sideInputStoreNames.contains(storeName);
         // Use the logged-store-base-directory for change logged stores and sideInput stores, and non-logged-store-base-dir
         // for non logged stores
diff --git a/samza-core/src/test/java/org/apache/samza/config/TestStorageConfig.java b/samza-core/src/test/java/org/apache/samza/config/TestStorageConfig.java
index e634940..ca7196a 100644
--- a/samza-core/src/test/java/org/apache/samza/config/TestStorageConfig.java
+++ b/samza-core/src/test/java/org/apache/samza/config/TestStorageConfig.java
@@ -146,9 +146,9 @@ public class TestStorageConfig {
     String factory3 = "factory3";
     StorageConfig storageConfig = new StorageConfig(new MapConfig(
         ImmutableMap.of(
-            String.format(STORE_BACKEND_BACKUP_FACTORIES, STORE_NAME0), factory1 + "," + factory2,
-            String.format(STORE_BACKEND_BACKUP_FACTORIES, STORE_NAME1), factory1,
-            String.format(STORE_BACKEND_BACKUP_FACTORIES, STORE_NAME2), factory3,
+            String.format(STORE_BACKUP_FACTORIES, STORE_NAME0), factory1 + "," + factory2,
+            String.format(STORE_BACKUP_FACTORIES, STORE_NAME1), factory1,
+            String.format(STORE_BACKUP_FACTORIES, STORE_NAME2), factory3,
             // store_name3 should use DEFAULT_STATE_BACKEND_FACTORY due to changelog presence
             String.format(CHANGELOG_STREAM, STORE_NAME3), "nondefault-changelog-system.streamName"),
         ImmutableMap.of(
@@ -158,20 +158,20 @@ public class TestStorageConfig {
             String.format(FACTORY, STORE_NAME3), "store3.factory.class",
             // this store should have no backend factory configured
             String.format(FACTORY, "noFactoryStore"), "noFactory.factory.class"
-            )
-        ));
-    Set<String> factories = storageConfig.getStateBackendBackupFactories();
+        )
+    ));
+    Set<String> factories = storageConfig.getBackupFactories();
     assertTrue(factories.contains(factory1));
     assertTrue(factories.contains(factory2));
     assertTrue(factories.contains(factory3));
-    assertTrue(factories.contains(DEFAULT_STATE_BACKEND_FACTORY));
+    assertTrue(factories.contains(KAFKA_STATE_BACKEND_FACTORY));
     assertEquals(4, factories.size());
-    assertEquals(ImmutableList.of(factory1, factory2), storageConfig.getStoreBackupManagerClassName(STORE_NAME0));
-    assertEquals(ImmutableList.of(factory1), storageConfig.getStoreBackupManagerClassName(STORE_NAME1));
-    assertEquals(ImmutableList.of(factory3), storageConfig.getStoreBackupManagerClassName(STORE_NAME2));
-    assertEquals(DEFAULT_STATE_BACKEND_BACKUP_FACTORIES, storageConfig.getStoreBackupManagerClassName(STORE_NAME3));
-    assertTrue(storageConfig.getStoreBackupManagerClassName("emptyStore").isEmpty());
-    assertTrue(storageConfig.getStoreBackupManagerClassName("noFactoryStore").isEmpty());
+    assertEquals(ImmutableList.of(factory1, factory2), storageConfig.getStoreBackupFactory(STORE_NAME0));
+    assertEquals(ImmutableList.of(factory1), storageConfig.getStoreBackupFactory(STORE_NAME1));
+    assertEquals(ImmutableList.of(factory3), storageConfig.getStoreBackupFactory(STORE_NAME2));
+    assertEquals(DEFAULT_BACKUP_FACTORIES, storageConfig.getStoreBackupFactory(STORE_NAME3));
+    assertTrue(storageConfig.getStoreBackupFactory("emptyStore").isEmpty());
+    assertTrue(storageConfig.getStoreBackupFactory("noFactoryStore").isEmpty());
   }
 
   @Test
@@ -179,10 +179,10 @@ public class TestStorageConfig {
     String targetFactory = "target.class";
     StorageConfig config = new StorageConfig(new MapConfig(
         ImmutableMap.of(
-            String.format(StorageConfig.STORE_BACKEND_BACKUP_FACTORIES, STORE_NAME0), targetFactory,
-            String.format(StorageConfig.STORE_BACKEND_BACKUP_FACTORIES, STORE_NAME1), targetFactory + "," +
-                DEFAULT_STATE_BACKEND_FACTORY,
-            String.format(StorageConfig.STORE_BACKEND_BACKUP_FACTORIES, STORE_NAME2), DEFAULT_STATE_BACKEND_FACTORY),
+            String.format(StorageConfig.STORE_BACKUP_FACTORIES, STORE_NAME0), targetFactory,
+            String.format(StorageConfig.STORE_BACKUP_FACTORIES, STORE_NAME1), targetFactory + "," +
+                KAFKA_STATE_BACKEND_FACTORY,
+            String.format(StorageConfig.STORE_BACKUP_FACTORIES, STORE_NAME2), KAFKA_STATE_BACKEND_FACTORY),
         ImmutableMap.of(
             String.format(FACTORY, STORE_NAME0), "store0.factory.class",
             String.format(FACTORY, STORE_NAME1), "store1.factory.class",
@@ -192,9 +192,9 @@ public class TestStorageConfig {
         )
     ));
 
-    List<String> targetStoreNames = config.getBackupStoreNamesForStateBackupFactory(targetFactory);
-    List<String> defaultStoreNames = config.getBackupStoreNamesForStateBackupFactory(
-        DEFAULT_STATE_BACKEND_FACTORY);
+    List<String> targetStoreNames = config.getStoresWithBackupFactory(targetFactory);
+    List<String> defaultStoreNames = config.getStoresWithBackupFactory(
+        KAFKA_STATE_BACKEND_FACTORY);
     assertTrue(targetStoreNames.containsAll(ImmutableList.of(STORE_NAME0, STORE_NAME1)));
     assertEquals(2, targetStoreNames.size());
     assertTrue(defaultStoreNames.containsAll(ImmutableList.of(STORE_NAME2, STORE_NAME1, STORE_NAME3)));
diff --git a/samza-core/src/test/java/org/apache/samza/storage/TestTaskStorageCommitManager.java b/samza-core/src/test/java/org/apache/samza/storage/TestTaskStorageCommitManager.java
index e682aef..ea00433 100644
--- a/samza-core/src/test/java/org/apache/samza/storage/TestTaskStorageCommitManager.java
+++ b/samza-core/src/test/java/org/apache/samza/storage/TestTaskStorageCommitManager.java
@@ -20,6 +20,7 @@
 package org.apache.samza.storage;
 
 import com.google.common.collect.ImmutableMap;
+
 import java.io.File;
 import java.io.FileFilter;
 import java.io.IOException;
@@ -47,10 +48,23 @@ import org.apache.samza.metrics.Timer;
 import org.apache.samza.system.SystemStream;
 import org.apache.samza.system.SystemStreamPartition;
 import org.junit.Test;
+import org.mockito.stubbing.Answer;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.*;
+import static org.mockito.Mockito.anyLong;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.doNothing;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
 
 
 public class TestTaskStorageCommitManager {
@@ -400,7 +414,7 @@ public class TestTaskStorageCommitManager {
     when(iStoreProps.isPersistedToDisk()).thenReturn(false);
     when(iStoreProps.isDurableStore()).thenReturn(false);
 
-    java.util.Map<String, StorageEngine> taskStores = ImmutableMap.of(
+    Map<String, StorageEngine> taskStores = ImmutableMap.of(
         "loggedPersistentStore", mockLPStore,
         "persistentStore", mockPStore,
         "loggedInMemStore", mockLIStore,
@@ -410,7 +424,7 @@ public class TestTaskStorageCommitManager {
     Partition changelogPartition = new Partition(0);
     SystemStream changelogSystemStream = new SystemStream("changelogSystem", "changelogStream");
     SystemStreamPartition changelogSSP = new SystemStreamPartition(changelogSystemStream, changelogPartition);
-    java.util.Map<String, SystemStream> storeChangelogsStreams = ImmutableMap.of(
+    Map<String, SystemStream> storeChangelogsStreams = ImmutableMap.of(
         "loggedPersistentStore", changelogSystemStream,
         "loggedInMemStore", new SystemStream("system", "stream")
     );
@@ -429,12 +443,17 @@ public class TestTaskStorageCommitManager {
         Collections.emptyMap(), containerStorageManager, storeChangelogsStreams, changelogPartition,
         null, null, ForkJoinPool.commonPool(), storageManagerUtil, durableStoreDir, metrics));
     doNothing().when(commitManager).writeChangelogOffsetFile(any(), any(), any(), any());
-
+    when(storageManagerUtil.getStoreCheckpointDir(any(File.class), any(CheckpointId.class)))
+        .thenAnswer((Answer<String>) invocation -> {
+          File file = invocation.getArgumentAt(0, File.class);
+          CheckpointId checkpointId = invocation.getArgumentAt(1, CheckpointId.class);
+          return file + "-" + checkpointId;
+        });
     CheckpointId newCheckpointId = CheckpointId.create();
 
     String newestOffset = "1";
     KafkaChangelogSSPOffset kafkaChangelogSSPOffset = new KafkaChangelogSSPOffset(newCheckpointId, newestOffset);
-    java.util.Map<SystemStreamPartition, String> offsetsJava = ImmutableMap.of(
+    Map<SystemStreamPartition, String> offsetsJava = ImmutableMap.of(
         changelogSSP, kafkaChangelogSSPOffset.toString()
     );
 
@@ -446,12 +465,12 @@ public class TestTaskStorageCommitManager {
     // evoked twice, for OFFSET-V1 and OFFSET-V2
     verify(commitManager).writeChangelogOffsetFile(
         eq("loggedPersistentStore"), eq(changelogSSP), eq(newestOffset), eq(durableStoreDir));
-    File checkpointFile = Paths.get(StorageManagerUtil
-        .getCheckpointDirPath(durableStoreDir, kafkaChangelogSSPOffset.getCheckpointId())).toFile();
+    File checkpointFile = Paths.get(storageManagerUtil.getStoreCheckpointDir(
+        durableStoreDir, kafkaChangelogSSPOffset.getCheckpointId())).toFile();
     verify(commitManager).writeChangelogOffsetFile(
         eq("loggedPersistentStore"), eq(changelogSSP), eq(newestOffset), eq(checkpointFile));
 
-    java.util.Map<String, String> storeSCM = ImmutableMap.of(
+    Map<String, String> storeSCM = ImmutableMap.of(
         "loggedPersistentStore", "system;loggedPersistentStoreStream;1",
         "persistentStore", "system;persistentStoreStream;1",
         "loggedInMemStore", "system;loggedInMemStoreStream;1",
@@ -464,7 +483,7 @@ public class TestTaskStorageCommitManager {
     // Validate only durable and persisted stores are persisted
     // This should be evoked twice, for checkpointV1 and checkpointV2
     verify(storageManagerUtil, times(2)).getTaskStoreDir(eq(durableStoreDir), eq("loggedPersistentStore"), eq(taskName), any());
-    File checkpointPath = Paths.get(StorageManagerUtil.getCheckpointDirPath(durableStoreDir, newCheckpointId)).toFile();
+    File checkpointPath = Paths.get(storageManagerUtil.getStoreCheckpointDir(durableStoreDir, newCheckpointId)).toFile();
     verify(storageManagerUtil).writeCheckpointV2File(eq(checkpointPath), eq(checkpoint));
   }
 
@@ -528,6 +547,13 @@ public class TestTaskStorageCommitManager {
         null, null, ForkJoinPool.commonPool(), storageManagerUtil, durableStoreDir, metrics));
     doNothing().when(commitManager).writeChangelogOffsetFile(any(), any(), any(), any());
 
+    when(storageManagerUtil.getStoreCheckpointDir(any(File.class), any(CheckpointId.class)))
+        .thenAnswer((Answer<String>) invocation -> {
+          File file = invocation.getArgumentAt(0, File.class);
+          CheckpointId checkpointId = invocation.getArgumentAt(1, CheckpointId.class);
+          return file + "-" + checkpointId;
+        });
+
     CheckpointId newCheckpointId = CheckpointId.create();
 
     java.util.Map<String, String> storeSCM = ImmutableMap.of(
@@ -543,7 +569,7 @@ public class TestTaskStorageCommitManager {
     commitManager.writeCheckpointToStoreDirectories(checkpoint);
     // Validate only durable and persisted stores are persisted
     verify(storageManagerUtil).getTaskStoreDir(eq(durableStoreDir), eq("loggedPersistentStore"), eq(taskName), any());
-    File checkpointPath = Paths.get(StorageManagerUtil.getCheckpointDirPath(durableStoreDir, newCheckpointId)).toFile();
+    File checkpointPath = Paths.get(storageManagerUtil.getStoreCheckpointDir(durableStoreDir, newCheckpointId)).toFile();
     verify(storageManagerUtil).writeCheckpointV2File(eq(checkpointPath), eq(checkpoint));
   }
 
@@ -579,6 +605,13 @@ public class TestTaskStorageCommitManager {
         Collections.emptyMap(), containerStorageManager, storeChangelogsStreams, changelogPartition,
         null, null, ForkJoinPool.commonPool(), storageManagerUtil, tmpTestPath, metrics));
 
+    when(storageManagerUtil.getStoreCheckpointDir(any(File.class), any(CheckpointId.class)))
+        .thenAnswer((Answer<String>) invocation -> {
+          File file = invocation.getArgumentAt(0, File.class);
+          CheckpointId checkpointId = invocation.getArgumentAt(1, CheckpointId.class);
+          return file + "-" + checkpointId;
+        });
+
     doAnswer(invocation -> {
       String fileDir = invocation.getArgumentAt(3, File.class).getName();
       SystemStreamPartition ssp = invocation.getArgumentAt(1, SystemStreamPartition.class);
@@ -607,8 +640,7 @@ public class TestTaskStorageCommitManager {
 
     assertEquals(2, mockFileSystem.size());
     // check if v2 offsets are written correctly
-    String v2FilePath = StorageManagerUtil
-        .getCheckpointDirPath(tmpTestPath, newCheckpointId);
+    String v2FilePath = storageManagerUtil.getStoreCheckpointDir(tmpTestPath, newCheckpointId);
     assertTrue(mockFileSystem.containsKey(v2FilePath));
     assertTrue(mockFileSystem.get(v2FilePath).containsKey(changelogSSP));
     assertEquals(1, mockFileSystem.get(v2FilePath).size());
@@ -675,6 +707,13 @@ public class TestTaskStorageCommitManager {
       return null;
     }).when(storageManagerUtil).writeCheckpointV2File(any(), any());
 
+    when(storageManagerUtil.getStoreCheckpointDir(any(File.class), any(CheckpointId.class)))
+        .thenAnswer((Answer<String>) invocation -> {
+          File file = invocation.getArgumentAt(0, File.class);
+          CheckpointId checkpointId = invocation.getArgumentAt(1, CheckpointId.class);
+          return file + "-" + checkpointId;
+        });
+
     CheckpointId newCheckpointId = CheckpointId.create();
 
     String newestOffset = "1";
@@ -689,8 +728,7 @@ public class TestTaskStorageCommitManager {
 
     assertEquals(2, mockFileSystem.size());
     // check if v2 offsets are written correctly
-    String v2FilePath = StorageManagerUtil
-        .getCheckpointDirPath(tmpTestPath, newCheckpointId);
+    String v2FilePath = storageManagerUtil.getStoreCheckpointDir(tmpTestPath, newCheckpointId);
     assertTrue(mockFileSystem.containsKey(v2FilePath));
     assertTrue(mockFileSystem.get(v2FilePath).containsKey(changelogSSP));
     assertEquals(1, mockFileSystem.get(v2FilePath).size());
diff --git a/samza-core/src/test/java/org/apache/samza/storage/blobstore/TestBlobStoreBackupManager.java b/samza-core/src/test/java/org/apache/samza/storage/blobstore/TestBlobStoreBackupManager.java
new file mode 100644
index 0000000..ca9e211
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/storage/blobstore/TestBlobStoreBackupManager.java
@@ -0,0 +1,542 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.storage.blobstore;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.util.concurrent.MoreExecutors;
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionStage;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.stream.Collectors;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.samza.SamzaException;
+import org.apache.samza.checkpoint.Checkpoint;
+import org.apache.samza.checkpoint.CheckpointId;
+import org.apache.samza.checkpoint.CheckpointV1;
+import org.apache.samza.checkpoint.CheckpointV2;
+import org.apache.samza.config.Config;
+import org.apache.samza.config.MapConfig;
+import org.apache.samza.container.TaskName;
+import org.apache.samza.job.model.ContainerModel;
+import org.apache.samza.job.model.JobModel;
+import org.apache.samza.job.model.TaskMode;
+import org.apache.samza.job.model.TaskModel;
+import org.apache.samza.metrics.Counter;
+import org.apache.samza.metrics.Gauge;
+import org.apache.samza.metrics.MetricsRegistry;
+import org.apache.samza.metrics.Timer;
+import org.apache.samza.storage.StorageEngine;
+import org.apache.samza.storage.StorageManagerUtil;
+import org.apache.samza.storage.blobstore.diff.DirDiff;
+import org.apache.samza.storage.blobstore.index.DirIndex;
+import org.apache.samza.storage.blobstore.index.SnapshotIndex;
+import org.apache.samza.storage.blobstore.index.SnapshotMetadata;
+import org.apache.samza.storage.blobstore.metrics.BlobStoreBackupManagerMetrics;
+import org.apache.samza.storage.blobstore.util.BlobStoreTestUtil;
+import org.apache.samza.storage.blobstore.util.BlobStoreUtil;
+import org.apache.samza.storage.blobstore.util.DirDiffUtil;
+import org.apache.samza.util.Clock;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import static org.mockito.Matchers.*;
+import static org.mockito.Mockito.anyLong;
+import static org.mockito.Mockito.eq;
+import static org.mockito.Mockito.*;
+
+
+public class TestBlobStoreBackupManager {
+  private final ExecutorService mockExecutor = MoreExecutors.newDirectExecutorService();
+  // mock container - task - job models
+  private final JobModel jobModel = mock(JobModel.class);
+  private final ContainerModel containerModel = mock(ContainerModel.class);
+  private final TaskModel taskModel = mock(TaskModel.class, RETURNS_DEEP_STUBS);
+  private final Clock clock = mock(Clock.class);
+  private final BlobStoreUtil blobStoreUtil = mock(BlobStoreUtil.class);
+  private final BlobStoreManager blobStoreManager = mock(BlobStoreManager.class);
+  private final StorageManagerUtil storageManagerUtil = mock(StorageManagerUtil.class);
+
+  //job and store definition
+  private final CheckpointId checkpointId = CheckpointId.deserialize("1234-567");
+  private final String jobName = "testJobName";
+  private final String jobId = "testJobID";
+  private final String taskName = "testTaskName";
+  private final String prevSnapshotIndexBlobId = "testPrevBlobId";
+  private Map<String, StorageEngine> storeStorageEngineMap = new HashMap<>();
+  private Map<String, String> mapConfig = new HashMap<>();
+
+  private final MetricsRegistry metricsRegistry = mock(MetricsRegistry.class);
+  private final Counter counter = mock(Counter.class);
+  private final Timer timer = mock(Timer.class);
+  private final Gauge<Long> longGauge = mock(Gauge.class);
+  private final Gauge<AtomicLong> atomicLongGauge = mock(Gauge.class);
+
+  private BlobStoreBackupManager blobStoreBackupManager;
+  private BlobStoreBackupManagerMetrics blobStoreTaskBackupMetrics;
+
+  // Remote and local snapshot definitions
+  private Map<String, SnapshotIndex> testBlobStore = new HashMap<>();
+  private Map<String, Pair<String, SnapshotIndex>> indexBlobIdAndLocalRemoteSnapshotsPair;
+  private Map<String, String> testStoreNameAndSCMMap;
+
+  @Before
+  public void setup() throws Exception {
+    when(clock.currentTimeMillis()).thenReturn(1234567L);
+    // setup test local and remote snapshots
+    indexBlobIdAndLocalRemoteSnapshotsPair = setupRemoteAndLocalSnapshots(true);
+    // setup test store name and SCMs map
+    testStoreNameAndSCMMap = setupTestStoreSCMMapAndStoreBackedFactoryConfig(indexBlobIdAndLocalRemoteSnapshotsPair);
+    // setup: setup task backup manager with expected storeName->storageEngine map
+    testStoreNameAndSCMMap.forEach((storeName, scm) -> storeStorageEngineMap.put(storeName, null));
+
+    mapConfig.putAll(new MapConfig(ImmutableMap.of("job.name", jobName, "job.id", jobId)));
+
+    Config config = new MapConfig(mapConfig);
+
+    // Mock - return snapshot index for blob id from test blob store map
+    ArgumentCaptor<String> captor = ArgumentCaptor.forClass(String.class);
+    when(blobStoreUtil.getSnapshotIndex(captor.capture(), any(Metadata.class)))
+        .then((Answer<CompletableFuture<SnapshotIndex>>) invocation -> {
+          String blobId = invocation.getArgumentAt(0, String.class);
+          return CompletableFuture.completedFuture(testBlobStore.get(blobId));
+        });
+
+//    doNothing().when(blobStoreManager).init();
+    when(taskModel.getTaskName().getTaskName()).thenReturn(taskName);
+    when(taskModel.getTaskMode()).thenReturn(TaskMode.Active);
+
+    when(metricsRegistry.newCounter(anyString(), anyString())).thenReturn(counter);
+    when(metricsRegistry.newGauge(anyString(), anyString(), anyLong())).thenReturn(longGauge);
+    when(metricsRegistry.newGauge(anyString(), anyString(), any(AtomicLong.class))).thenReturn(atomicLongGauge);
+    when(atomicLongGauge.getValue()).thenReturn(new AtomicLong());
+    when(metricsRegistry.newTimer(anyString(), anyString())).thenReturn(timer);
+    blobStoreTaskBackupMetrics = new BlobStoreBackupManagerMetrics(metricsRegistry);
+
+    blobStoreBackupManager =
+        new MockBlobStoreBackupManager(jobModel, containerModel, taskModel, mockExecutor,
+            blobStoreTaskBackupMetrics, config,
+            Files.createTempDirectory("logged-store-").toFile(), storageManagerUtil, blobStoreManager);
+  }
+
+  @Test
+  public void testInitWithInvalidCheckpointFails() {
+    // init called with null checkpoint storeStorageEngineMap
+    blobStoreBackupManager.init(null);
+    // verify delete snapshot index blob called from init 0 times because prevSnapshotMap returned from init is empty
+    // in case of null checkpoint.
+    verify(blobStoreUtil, times(0)).deleteSnapshotIndexBlob(anyString(), any(Metadata.class));
+    when(blobStoreUtil.getStoreSnapshotIndexes(anyString(), anyString(), anyString(), any(Checkpoint.class))).thenCallRealMethod();
+
+    // init called with Checkpoint V1 -> unsupported
+    Checkpoint checkpoint = new CheckpointV1(new HashMap<>());
+    String expectedException = "Checkpoint version 1 is not supported for blob store backup and restore.";
+    try {
+      blobStoreBackupManager.init(checkpoint);
+      Assert.fail("Checkpoint V1 is exepcted to fail.");
+    } catch (SamzaException exception) {
+      Assert.assertEquals(exception.getMessage(), expectedException);
+    }
+  }
+
+  @Test
+  public void testUploadWithNoPreviousCheckpoints() throws IOException {
+    // Track directory for post cleanup
+    List<String> checkpointDirsToClean = new ArrayList<>();
+
+    // Setup: init local/remote snapshots and back manager with no previous checkpoints
+    indexBlobIdAndLocalRemoteSnapshotsPair = setupRemoteAndLocalSnapshots(false);
+    Checkpoint checkpoint =
+        new CheckpointV2(checkpointId, new HashMap<>(),
+            ImmutableMap.of(BlobStoreStateBackendFactory.class.getName(), new HashMap<>()));
+    blobStoreBackupManager.init(checkpoint);
+
+    // mock: set task store dir to return corresponding test local store and create checkpoint dir
+    ArgumentCaptor<String> stringCaptor = ArgumentCaptor.forClass(String.class);
+    when(storageManagerUtil.getTaskStoreDir(any(File.class), stringCaptor.capture(), any(TaskName.class), any(TaskMode.class)))
+        .then((Answer<File>) invocation -> {
+          String storeName = invocation.getArgumentAt(1, String.class);
+          String snapshotIndexBlobId = testStoreNameAndSCMMap.get(storeName);
+          String storeDir = indexBlobIdAndLocalRemoteSnapshotsPair.get(snapshotIndexBlobId).getLeft();
+          try {
+            BlobStoreTestUtil.createTestCheckpointDirectory(storeDir, checkpointId.serialize()); // create test checkpoint dir
+            checkpointDirsToClean.add(storeDir + "-" + checkpointId.serialize()); // track checkpoint dir to cleanup later
+          } catch (IOException e) {
+            Assert.fail("Couldn't create checkpoint directory. Test failed.");
+          }
+          return new File(storeDir);
+        });
+
+
+    ArgumentCaptor<File> storeDirCaptor = ArgumentCaptor.forClass(File.class);
+    when(storageManagerUtil.getStoreCheckpointDir(storeDirCaptor.capture(), eq(checkpointId)))
+        .thenAnswer(new Answer<String>() {
+          @Override
+          public String answer(InvocationOnMock invocation) throws Throwable {
+            File storeDir = invocation.getArgumentAt(0, File.class);
+            return storeDir.getAbsolutePath() + "-" + checkpointId.serialize();
+          }
+        });
+
+    SortedSet<DirDiff> actualDirDiffs = new TreeSet<>(Comparator.comparing(DirDiff::getDirName));
+    // mock: mock putDir and capture DirDiff
+    ArgumentCaptor<DirDiff> dirDiffCaptor = ArgumentCaptor.forClass(DirDiff.class);
+    ArgumentCaptor<SnapshotMetadata> snapshotMetadataCaptor = ArgumentCaptor.forClass(SnapshotMetadata.class);
+    when(blobStoreUtil.putDir(dirDiffCaptor.capture(), snapshotMetadataCaptor.capture()))
+        .then((Answer<CompletableFuture<DirIndex>>) invocation -> {
+          DirDiff dirDiff = invocation.getArgumentAt(0, DirDiff.class);
+          SnapshotMetadata snapshotMetadata = invocation.getArgumentAt(1, SnapshotMetadata.class);
+          actualDirDiffs.add(dirDiff);
+          SnapshotIndex snapshotIndex = testBlobStore.get(testStoreNameAndSCMMap.get(snapshotMetadata.getStoreName()));
+          return CompletableFuture.completedFuture(snapshotIndex.getDirIndex());
+        });
+
+    SortedSet<SnapshotIndex> expectedSnapshotIndexesUploaded = indexBlobIdAndLocalRemoteSnapshotsPair.values().stream()
+        .map(Pair::getRight)
+        .collect(Collectors.toCollection(() -> new TreeSet<>(Comparator.comparing(SnapshotIndex::getCreationTimeMillis))));
+    String expectedPreviousSnapshotIndexBlobId = "empty";
+    // mock: mock putSnapshotIndex and capture previous snapshot index
+    SortedSet<SnapshotIndex> actualSnapshotIndexesUploaded =
+        new TreeSet<>(Comparator.comparing(SnapshotIndex::getCreationTimeMillis));
+    final String[] actualPreviousSnapshotIndexBlobId = {"empty"};
+    ArgumentCaptor<SnapshotIndex> snapshotIndexCaptor = ArgumentCaptor.forClass(SnapshotIndex.class);
+    when(blobStoreUtil.putSnapshotIndex(snapshotIndexCaptor.capture()))
+        .then((Answer<CompletableFuture<String>>) invocation -> {
+          SnapshotIndex snapshotIndex = invocation.getArgumentAt(0, SnapshotIndex.class);
+          actualSnapshotIndexesUploaded.add(snapshotIndex);
+          if (!snapshotIndex.getPrevSnapshotIndexBlobId().equals(Optional.empty())) {
+            actualPreviousSnapshotIndexBlobId[0] = "not-empty";
+          }
+          return CompletableFuture.completedFuture("random-blob-id");
+        });
+
+    // execute
+    blobStoreBackupManager.upload(checkpointId, testStoreNameAndSCMMap);
+
+    // setup expected dir diffs after execute: needs checkpoint dirs created in upload()
+    TreeSet<DirDiff> expectedDirDiffs = indexBlobIdAndLocalRemoteSnapshotsPair.values().stream()
+        .map(localRemoteSnapshotPair -> {
+          File localCheckpointDir = new File(localRemoteSnapshotPair.getLeft() + "-" + checkpointId.serialize());
+          DirIndex dirIndex = new DirIndex(localCheckpointDir.getName(), Collections.emptyList(), Collections.emptyList(),
+              Collections.emptyList(), Collections.emptyList());
+          return DirDiffUtil.getDirDiff(localCheckpointDir, dirIndex, DirDiffUtil.areSameFile(false));
+        }).collect(Collectors.toCollection(() -> new TreeSet<>(Comparator.comparing(DirDiff::getDirName))));
+
+    // assert - asset all DirDiff are put to blob store
+    Assert.assertEquals(actualDirDiffs, expectedDirDiffs);
+    // assert - assert no previous snapshot indexes were found
+    Assert.assertEquals(actualPreviousSnapshotIndexBlobId[0], expectedPreviousSnapshotIndexBlobId);
+    // assert - assert all snapshot indexes are uploaded
+    Assert.assertEquals(actualSnapshotIndexesUploaded, expectedSnapshotIndexesUploaded);
+
+    // cleanup
+    checkpointDirsToClean.forEach(path -> {
+      try {
+        FileUtils.deleteDirectory(new File(path));
+      } catch (IOException exception) {
+        Assert.fail("Failed to cleanup temporary checkpoint dirs.");
+      }
+    });
+  }
+
+  @Test
+  public void testUploadWithPreviousCheckpoints() throws IOException {
+    // Track directory for post cleanup
+    List<String> checkpointDirsToClean = new ArrayList<>();
+
+    // Setup: init back manager with previous checkpoints
+    //indexBlobIdAndLocalRemoteSnapshotsPair = setupRemoteAndLocalSnapshots(true);
+    Map<String, String> previousCheckpoints =
+        // map store name, previous snapshot index blob id
+        indexBlobIdAndLocalRemoteSnapshotsPair.entrySet().stream()
+            .collect(Collectors.toMap(e -> e.getValue().getLeft(),
+              e -> e.getValue().getRight().getPrevSnapshotIndexBlobId().get()));
+
+    Checkpoint checkpoint =
+        new CheckpointV2(checkpointId, new HashMap<>(),
+            ImmutableMap.of(BlobStoreStateBackendFactory.class.getName(), previousCheckpoints));
+    when(blobStoreUtil.getStoreSnapshotIndexes(anyString(), anyString(), anyString(), any(Checkpoint.class))).thenCallRealMethod();
+    blobStoreBackupManager.init(checkpoint);
+
+    // mock: set task store dir to return corresponding test local store and create checkpoint dir
+    ArgumentCaptor<String> stringCaptor = ArgumentCaptor.forClass(String.class);
+    when(storageManagerUtil.getTaskStoreDir(any(File.class), stringCaptor.capture(), any(TaskName.class), any(TaskMode.class)))
+        .then((Answer<File>) invocation -> {
+          String storeName = invocation.getArgumentAt(1, String.class);
+          String snapshotIndexBlobId = testStoreNameAndSCMMap.get(storeName);
+          String storeDir = indexBlobIdAndLocalRemoteSnapshotsPair.get(snapshotIndexBlobId).getLeft();
+          try { // create test checkpoint dir
+            BlobStoreTestUtil.createTestCheckpointDirectory(storeDir, checkpointId.serialize());
+            checkpointDirsToClean.add(storeDir + "-" + checkpointId.serialize());
+          } catch (IOException e) {
+            Assert.fail("Couldn't create checkpoint directory. Test failed.");
+          }
+          return new File(storeDir);
+        });
+
+    ArgumentCaptor<File> storeDirCaptor = ArgumentCaptor.forClass(File.class);
+    when(storageManagerUtil.getStoreCheckpointDir(storeDirCaptor.capture(), eq(checkpointId)))
+        .thenAnswer(new Answer<String>() {
+          @Override
+          public String answer(InvocationOnMock invocation) throws Throwable {
+            File storeDir = invocation.getArgumentAt(0, File.class);
+            return storeDir.getAbsolutePath() + "-" + checkpointId.serialize();
+          }
+        });
+
+    // mock: mock putDir and capture DirDiff
+    SortedSet<DirDiff> actualDirDiffs = new TreeSet<>(Comparator.comparing(DirDiff::getDirName));
+    ArgumentCaptor<DirDiff> dirDiffCaptor = ArgumentCaptor.forClass(DirDiff.class);
+    ArgumentCaptor<SnapshotMetadata> snapshotMetadataCaptor = ArgumentCaptor.forClass(SnapshotMetadata.class);
+    when(blobStoreUtil.putDir(dirDiffCaptor.capture(), snapshotMetadataCaptor.capture()))
+        .then((Answer<CompletableFuture<DirIndex>>) invocation -> {
+          DirDiff dirDiff = invocation.getArgumentAt(0, DirDiff.class);
+          SnapshotMetadata snapshotMetadata = invocation.getArgumentAt(1, SnapshotMetadata.class);
+          actualDirDiffs.add(dirDiff);
+          SnapshotIndex snapshotIndex = testBlobStore.get(testStoreNameAndSCMMap.get(snapshotMetadata.getStoreName()));
+          return CompletableFuture.completedFuture(snapshotIndex.getDirIndex());
+        });
+
+    // mock: mock putSnapshotIndex and capture previous snapshot index
+    SortedSet<SnapshotIndex> expectedSnapshotIndexesUploaded = indexBlobIdAndLocalRemoteSnapshotsPair.values().stream()
+        .map(Pair::getRight)
+        .collect(Collectors.toCollection(() -> new TreeSet<>(Comparator.comparing(SnapshotIndex::getCreationTimeMillis))));
+    SortedSet<SnapshotIndex> actualSnapshotIndexesUploaded = new TreeSet<>(Comparator.comparing(SnapshotIndex::getCreationTimeMillis));
+    SortedSet<String> actualPreviousSnapshotIndexBlobIds = new TreeSet<>();
+    SortedSet<String> expectedPreviousSnapshotIndexBlobIds = new TreeSet<>(previousCheckpoints.values());
+    ArgumentCaptor<SnapshotIndex> snapshotIndexCaptor = ArgumentCaptor.forClass(SnapshotIndex.class);
+    when(blobStoreUtil.putSnapshotIndex(snapshotIndexCaptor.capture()))
+        .then((Answer<CompletableFuture<String>>) invocation -> {
+          SnapshotIndex snapshotIndex = invocation.getArgumentAt(0, SnapshotIndex.class);
+          actualSnapshotIndexesUploaded.add(snapshotIndex);
+          if (snapshotIndex.getPrevSnapshotIndexBlobId().isPresent()) {
+            actualPreviousSnapshotIndexBlobIds.add(snapshotIndex.getPrevSnapshotIndexBlobId().get());
+          }
+          return CompletableFuture.completedFuture("random-blob-id");
+        });
+
+    // execute
+    blobStoreBackupManager.upload(checkpointId, ImmutableMap.of());
+
+    TreeSet<DirDiff> expectedDirDiffs = indexBlobIdAndLocalRemoteSnapshotsPair.values()
+        .stream()
+        .map(localRemoteSnapshotPair ->
+            DirDiffUtil.getDirDiff(new File(localRemoteSnapshotPair.getLeft() + "-" + checkpointId.serialize()),
+            localRemoteSnapshotPair.getRight().getDirIndex(), DirDiffUtil.areSameFile(false)))
+        .collect(Collectors.toCollection(() -> new TreeSet<>(Comparator.comparing(DirDiff::getDirName))));
+
+    // assert - asset all DirDiff are put to blob store
+    Assert.assertEquals(actualDirDiffs, expectedDirDiffs);
+    // assert - assert no previous snapshot indexes were found
+    Assert.assertEquals(actualPreviousSnapshotIndexBlobIds, expectedPreviousSnapshotIndexBlobIds);
+    // assert - assert all snapshot indexes are uploaded
+    Assert.assertEquals(actualSnapshotIndexesUploaded, expectedSnapshotIndexesUploaded);
+
+    // cleanup
+    checkpointDirsToClean.forEach(path -> {
+      try {
+        FileUtils.deleteDirectory(new File(path));
+      } catch (IOException exception) {
+        Assert.fail("Failed to cleanup temporary checkpoint dirs.");
+      }
+    });
+  }
+
+
+
+  @Test
+  public void testCleanupRemovesTTLForAllIndexBlobs() {
+    SortedSet<String> actualRemoveTTLsResult = new TreeSet<>(testStoreNameAndSCMMap.values());
+
+    SortedSet<String> expectedRemoveTTLsResult = new TreeSet<>();
+
+    // mock
+    ArgumentCaptor<String> captor = ArgumentCaptor.forClass(String.class);
+    when(blobStoreUtil.removeTTL(captor.capture(), any(SnapshotIndex.class), any(Metadata.class)))
+        .then((Answer<CompletionStage<Void>>) invocation -> {
+          String blobId = invocation.getArgumentAt(0, String.class);
+          expectedRemoveTTLsResult.add(blobId);
+          return CompletableFuture.completedFuture(null);
+        });
+
+    // stub out non-tested methods
+    when(blobStoreUtil.cleanUpDir(any(DirIndex.class), any(Metadata.class))).thenReturn(CompletableFuture.completedFuture(null));
+    when(blobStoreUtil.deleteSnapshotIndexBlob(any(String.class), any(Metadata.class))).thenReturn(CompletableFuture.completedFuture(null));
+
+    // execute
+    blobStoreBackupManager.cleanUp(checkpointId, testStoreNameAndSCMMap);
+
+    // Assert
+    Assert.assertEquals(actualRemoveTTLsResult, expectedRemoveTTLsResult);
+  }
+
+  @Test
+  public void testCleanupCleansUpRemoteSnapshot() throws Exception {
+    SortedSet<DirIndex> actualCleanedupDirs = indexBlobIdAndLocalRemoteSnapshotsPair.values().stream()
+        .map(remoteLocalPair -> remoteLocalPair.getRight().getDirIndex())
+        .collect(Collectors.toCollection(() -> new TreeSet<>(Comparator.comparing(DirIndex::getDirName))));
+
+    SortedSet<DirIndex> expectedCleanupDirs = new TreeSet<>(Comparator.comparing(DirIndex::getDirName));
+
+    // mock
+    ArgumentCaptor<DirIndex> captor = ArgumentCaptor.forClass(DirIndex.class);
+    when(blobStoreUtil.cleanUpDir(captor.capture(), any(Metadata.class)))
+        .then((Answer<CompletionStage<Void>>) invocation -> {
+          DirIndex dirIndex = invocation.getArgumentAt(0, DirIndex.class);
+          expectedCleanupDirs.add(dirIndex);
+          return CompletableFuture.completedFuture(null);
+        });
+
+    // stub out non-tested methods
+    when(blobStoreUtil.removeTTL(anyString(), any(SnapshotIndex.class), any(Metadata.class)))
+        .thenReturn(CompletableFuture.completedFuture(null));
+    when(blobStoreUtil.deleteSnapshotIndexBlob(any(String.class), any(Metadata.class)))
+        .thenReturn(CompletableFuture.completedFuture(null));
+
+    // execute
+    blobStoreBackupManager.cleanUp(checkpointId, testStoreNameAndSCMMap);
+
+    // Assert
+    Assert.assertEquals(actualCleanedupDirs, expectedCleanupDirs);
+  }
+
+  @Test
+  public void testCleanupRemovesOldSnapshots() throws Exception {
+    TreeSet<String> expectedOldSnapshotsRemoved = indexBlobIdAndLocalRemoteSnapshotsPair.values().stream()
+        .map(remoteLocalPair -> {
+          Optional<String> prevSnapshotIndexBlobId = remoteLocalPair.getRight().getPrevSnapshotIndexBlobId();
+          return prevSnapshotIndexBlobId.orElse(null);
+        })
+        .collect(Collectors.toCollection(TreeSet::new));
+
+    SortedSet<String> actualOldSnapshotsRemoved = new TreeSet<>();
+
+    // mock
+    ArgumentCaptor<String> captor = ArgumentCaptor.forClass(String.class);
+    when(blobStoreUtil.deleteSnapshotIndexBlob(captor.capture(), any(Metadata.class)))
+        .then((Answer<CompletionStage<Void>>) invocation -> {
+          String prevIndexBlobId = invocation.getArgumentAt(0, String.class);
+          actualOldSnapshotsRemoved.add(prevIndexBlobId);
+          return CompletableFuture.completedFuture(null);
+        });
+
+    // stub out non-tested methods
+    when(blobStoreUtil.removeTTL(anyString(), any(SnapshotIndex.class), any(Metadata.class)))
+        .thenReturn(CompletableFuture.completedFuture(null));
+    when(blobStoreUtil.cleanUpDir(any(DirIndex.class), any(Metadata.class)))
+        .thenReturn(CompletableFuture.completedFuture(null));
+
+    // execute
+    blobStoreBackupManager.cleanUp(checkpointId, testStoreNameAndSCMMap);
+
+    // Assert
+    Assert.assertEquals(actualOldSnapshotsRemoved, expectedOldSnapshotsRemoved);
+  }
+
+  @Test
+  public void testCleanupIgnoresStoresNotConfiguredWithBlobStoreStateBackend() throws Exception {
+    // TODO HIGH shesharm Complete test
+  }
+
+  private Map<String, String> setupTestStoreSCMMapAndStoreBackedFactoryConfig(Map<String,
+      Pair<String, SnapshotIndex>> indexBlobIdAndRemoteAndLocalSnapshotMap) {
+    Map<String, String> storeNameSCMMap = new HashMap<>();
+    indexBlobIdAndRemoteAndLocalSnapshotMap
+        .forEach((blobId, localRemoteSnapshots) -> {
+          mapConfig.put("stores." + localRemoteSnapshots.getLeft() + ".factory",
+              BlobStoreStateBackendFactory.class.getName());
+          mapConfig.put("stores." + localRemoteSnapshots.getLeft() + ".backup.factories",
+              BlobStoreStateBackendFactory.class.getName());
+          storeNameSCMMap.put(localRemoteSnapshots.getLeft(), blobId);
+        });
+    return storeNameSCMMap;
+  }
+
+  private Map<String, Pair<String, SnapshotIndex>> setupRemoteAndLocalSnapshots(boolean addPrevCheckpoints) throws IOException {
+    testBlobStore = new HashMap<>(); // reset blob store
+    Map<String, Pair<String, SnapshotIndex>> indexBlobIdAndRemoteAndLocalSnapshotMap = new HashMap<>();
+    List<String> localSnapshots = new ArrayList<>();
+    List<String> previousRemoteSnapshots = new ArrayList<>();
+
+    localSnapshots.add("[a, c, z/1, y/2, p/m/3, q/n/4]");
+    previousRemoteSnapshots.add("[a, b, z/1, x/5, p/m/3, r/o/6]");
+
+    localSnapshots.add("[a, c, z/1, y/1, p/m/1, q/n/1]");
+    previousRemoteSnapshots.add("[a, z/1, p/m/1]");
+
+    localSnapshots.add("[z/i/1, y/j/1]");
+    previousRemoteSnapshots.add("[z/i/1, x/k/1]");
+
+    // setup local and corresponding remote snapshots
+    for (int i = 0; i < localSnapshots.size(); i++) {
+      Path localSnapshot = BlobStoreTestUtil.createLocalDir(localSnapshots.get(i));
+      String testLocalSnapshot = localSnapshot.toAbsolutePath().toString();
+      DirIndex dirIndex = BlobStoreTestUtil.createDirIndex(localSnapshots.get(i));
+      SnapshotMetadata snapshotMetadata = new SnapshotMetadata(checkpointId, jobName, jobId, taskName, testLocalSnapshot);
+      Optional<String> prevCheckpointId = Optional.empty();
+      if (addPrevCheckpoints) {
+        prevCheckpointId = Optional.of(prevSnapshotIndexBlobId + "-" + i);
+        DirIndex prevDirIndex = BlobStoreTestUtil.createDirIndex(previousRemoteSnapshots.get(i));
+        testBlobStore.put(prevCheckpointId.get(),
+            new SnapshotIndex(clock.currentTimeMillis(), snapshotMetadata, prevDirIndex, Optional.empty()));
+      }
+      SnapshotIndex testRemoteSnapshot =
+          new SnapshotIndex(clock.currentTimeMillis(), snapshotMetadata, dirIndex, prevCheckpointId);
+      indexBlobIdAndRemoteAndLocalSnapshotMap.put("blobId-" + i, Pair.of(testLocalSnapshot, testRemoteSnapshot));
+      testBlobStore.put("blobId-" + i, testRemoteSnapshot);
+    }
+    return indexBlobIdAndRemoteAndLocalSnapshotMap;
+  }
+
+  private class MockBlobStoreBackupManager extends BlobStoreBackupManager {
+
+    public MockBlobStoreBackupManager(JobModel jobModel, ContainerModel containerModel, TaskModel taskModel,
+        ExecutorService backupExecutor, BlobStoreBackupManagerMetrics blobStoreTaskBackupMetrics, Config config,
+        File loggedStoreBaseDir, StorageManagerUtil storageManagerUtil,
+        BlobStoreManager blobStoreManager) {
+      super(jobModel, containerModel, taskModel, backupExecutor, blobStoreTaskBackupMetrics, config, clock,
+          loggedStoreBaseDir, storageManagerUtil, blobStoreManager);
+    }
+
+    @Override
+    protected BlobStoreUtil createBlobStoreUtil(BlobStoreManager blobStoreManager, ExecutorService executor,
+        BlobStoreBackupManagerMetrics metrics) {
+      return blobStoreUtil;
+    }
+  }
+}
diff --git a/samza-core/src/test/java/org/apache/samza/storage/blobstore/TestBlobStoreRestoreManager.java b/samza-core/src/test/java/org/apache/samza/storage/blobstore/TestBlobStoreRestoreManager.java
new file mode 100644
index 0000000..7e0da6d
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/storage/blobstore/TestBlobStoreRestoreManager.java
@@ -0,0 +1,361 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.storage.blobstore;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.util.concurrent.MoreExecutors;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.samza.checkpoint.CheckpointId;
+import org.apache.samza.config.BlobStoreConfig;
+import org.apache.samza.config.StorageConfig;
+import org.apache.samza.container.TaskName;
+import org.apache.samza.job.model.TaskMode;
+import org.apache.samza.metrics.MetricsRegistryMap;
+import org.apache.samza.storage.StorageManagerUtil;
+import org.apache.samza.storage.blobstore.index.DirIndex;
+import org.apache.samza.storage.blobstore.index.SnapshotIndex;
+import org.apache.samza.storage.blobstore.index.SnapshotMetadata;
+import org.apache.samza.storage.blobstore.metrics.BlobStoreRestoreManagerMetrics;
+import org.apache.samza.storage.blobstore.util.BlobStoreTestUtil;
+import org.apache.samza.storage.blobstore.util.BlobStoreUtil;
+import org.apache.samza.storage.blobstore.util.DirDiffUtil;
+import org.junit.Test;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyBoolean;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.anySet;
+import static org.mockito.Mockito.anyString;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyZeroInteractions;
+import static org.mockito.Mockito.when;
+
+
+public class TestBlobStoreRestoreManager {
+  private static final ExecutorService EXECUTOR = MoreExecutors.newDirectExecutorService();
+
+  @Test
+  public void testDeleteUnusedStoresRemovesStoresDeletedFromConfig() {
+    String jobName = "testJobName";
+    String jobId = "testJobId";
+    String taskName = "taskName";
+    StorageConfig storageConfig = mock(StorageConfig.class);
+    BlobStoreConfig blobStoreConfig = mock(BlobStoreConfig.class);
+    SnapshotIndex mockSnapshotIndex = mock(SnapshotIndex.class);
+    String blobId = "blobId";
+    Map<String, Pair<String, SnapshotIndex>> initialStoreSnapshotIndexes =
+        ImmutableMap.of("oldStoreName", Pair.of(blobId, mockSnapshotIndex));
+
+    when(storageConfig.getStoresWithBackupFactory(eq(BlobStoreStateBackendFactory.class.getName())))
+        .thenReturn(ImmutableList.of("newStoreName"));
+    when(storageConfig.getStoresWithRestoreFactory(eq(BlobStoreStateBackendFactory.class.getName())))
+        .thenReturn(ImmutableList.of("newStoreName"));
+
+    DirIndex dirIndex = mock(DirIndex.class);
+    when(mockSnapshotIndex.getDirIndex()).thenReturn(dirIndex);
+
+    BlobStoreUtil blobStoreUtil = mock(BlobStoreUtil.class);
+    when(blobStoreUtil.cleanUpDir(any(DirIndex.class), any(Metadata.class))).thenReturn(CompletableFuture.completedFuture(null));
+    when(blobStoreUtil.deleteDir(any(DirIndex.class), any(Metadata.class))).thenReturn(CompletableFuture.completedFuture(null));
+    when(blobStoreUtil.deleteSnapshotIndexBlob(anyString(), any(Metadata.class))).thenReturn(CompletableFuture.completedFuture(null));
+
+    BlobStoreRestoreManager.deleteUnusedStoresFromBlobStore(
+        jobName, jobId, taskName, storageConfig, blobStoreConfig, initialStoreSnapshotIndexes, blobStoreUtil, EXECUTOR);
+
+    verify(blobStoreUtil, times(1)).cleanUpDir(eq(dirIndex), any(Metadata.class));
+    verify(blobStoreUtil, times(1)).deleteDir(eq(dirIndex), any(Metadata.class));
+    verify(blobStoreUtil, times(1)).deleteSnapshotIndexBlob(eq(blobId), any(Metadata.class));
+
+  }
+
+  @Test
+  public void testShouldRestoreIfNoCheckpointDir() throws IOException {
+    String taskName = "taskName";
+    String storeName = "storeName";
+    DirIndex dirIndex = mock(DirIndex.class);
+    Path storeCheckpointDir = Paths.get("/tmp/non-existent-checkpoint-dir");
+    StorageConfig storageConfig = mock(StorageConfig.class);
+    when(storageConfig.cleanLoggedStoreDirsOnStart(anyString())).thenReturn(false);
+    DirDiffUtil dirDiffUtil = mock(DirDiffUtil.class);
+
+    boolean shouldRestore = BlobStoreRestoreManager.shouldRestore(
+        taskName, storeName, dirIndex, storeCheckpointDir, storageConfig, dirDiffUtil);
+
+    verifyZeroInteractions(dirDiffUtil);
+    assertTrue(shouldRestore);
+  }
+
+  @Test
+  public void testShouldRestoreIfCleanStateOnRestartEnabled() throws  IOException {
+    String taskName = "taskName";
+    String storeName = "storeName";
+    DirIndex dirIndex = mock(DirIndex.class);
+    Path storeCheckpointDir = Files.createTempDirectory(BlobStoreTestUtil.TEMP_DIR_PREFIX); // must exist
+    StorageConfig storageConfig = mock(StorageConfig.class);
+    when(storageConfig.cleanLoggedStoreDirsOnStart(anyString())).thenReturn(true); // clean on restart
+    DirDiffUtil dirDiffUtil = mock(DirDiffUtil.class);
+
+    boolean shouldRestore = BlobStoreRestoreManager.shouldRestore(
+        taskName, storeName, dirIndex, storeCheckpointDir, storageConfig, dirDiffUtil);
+
+    verifyZeroInteractions(dirDiffUtil);
+    assertTrue(shouldRestore); // should not restore, should retain checkpoint dir instead
+  }
+
+  @Test
+  public void testShouldRestoreIfCheckpointDirNotIdenticalToRemoteSnapshot() throws IOException {
+    String taskName = "taskName";
+    String storeName = "storeName";
+    DirIndex dirIndex = mock(DirIndex.class);
+    Path storeCheckpointDir = Files.createTempDirectory(BlobStoreTestUtil.TEMP_DIR_PREFIX); // must exist
+    StorageConfig storageConfig = mock(StorageConfig.class);
+    when(storageConfig.cleanLoggedStoreDirsOnStart(anyString())).thenReturn(false);
+    DirDiffUtil dirDiffUtil = mock(DirDiffUtil.class);
+    when(dirDiffUtil.areSameDir(anySet(), anyBoolean())).thenReturn((arg1, arg2) -> false);
+
+    boolean shouldRestore = BlobStoreRestoreManager.shouldRestore(
+        taskName, storeName, dirIndex, storeCheckpointDir, storageConfig, dirDiffUtil);
+
+    assertTrue(shouldRestore);
+  }
+
+  @Test
+  public void testShouldNotRestoreIfPreviousCheckpointDirIdenticalToRemoteSnapshot() throws  IOException {
+    String taskName = "taskName";
+    String storeName = "storeName";
+    DirIndex dirIndex = mock(DirIndex.class);
+    Path storeCheckpointDir = Files.createTempDirectory(BlobStoreTestUtil.TEMP_DIR_PREFIX); // must exist
+    StorageConfig storageConfig = mock(StorageConfig.class);
+    when(storageConfig.cleanLoggedStoreDirsOnStart(anyString())).thenReturn(false);
+    DirDiffUtil dirDiffUtil = mock(DirDiffUtil.class);
+    when(dirDiffUtil.areSameDir(anySet(), anyBoolean())).thenReturn((arg1, arg2) -> true); // are same dir
+
+    boolean shouldRestore = BlobStoreRestoreManager.shouldRestore(
+        taskName, storeName, dirIndex, storeCheckpointDir, storageConfig, dirDiffUtil);
+
+    verify(dirDiffUtil, times(1)).areSameDir(anySet(), anyBoolean());
+    assertFalse(shouldRestore); // should not restore, should retain checkpoint dir instead
+  }
+
+  @Test
+  public void testRestoreDeletesStoreDir() throws IOException {
+    String jobName = "testJobName";
+    String jobId = "testJobId";
+    TaskName taskName = mock(TaskName.class);
+    BlobStoreRestoreManagerMetrics metrics = new BlobStoreRestoreManagerMetrics(new MetricsRegistryMap());
+    metrics.initStoreMetrics(ImmutableList.of("storeName"));
+    List<String> storesToRestore = ImmutableList.of("storeName");
+    SnapshotIndex snapshotIndex = mock(SnapshotIndex.class);
+    Map<String, Pair<String, SnapshotIndex>> prevStoreSnapshotIndexes =
+        ImmutableMap.of("storeName", Pair.of("blobId", snapshotIndex));
+    DirIndex dirIndex = BlobStoreTestUtil.createDirIndex("[a]");
+    when(snapshotIndex.getDirIndex()).thenReturn(dirIndex);
+    when(snapshotIndex.getSnapshotMetadata())
+        .thenReturn(new SnapshotMetadata(CheckpointId.create(), "jobName", "jobId", "taskName", "storeName"));
+
+    Path loggedBaseDir = Files.createTempDirectory(BlobStoreTestUtil.TEMP_DIR_PREFIX);
+
+    // create store dir to be deleted during restore
+    Path storeDir = Files.createTempDirectory(loggedBaseDir, "storeDir");
+    StorageConfig storageConfig = mock(StorageConfig.class);
+    StorageManagerUtil storageManagerUtil = mock(StorageManagerUtil.class);
+    when(storageManagerUtil.getStoreCheckpointDir(any(File.class), any(CheckpointId.class)))
+      .thenReturn(Paths.get(storeDir.toString(), "checkpointId").toString());
+    when(storageManagerUtil.getTaskStoreDir(
+        eq(loggedBaseDir.toFile()), eq("storeName"), eq(taskName), eq(TaskMode.Active)))
+          .thenReturn(storeDir.toFile());
+    BlobStoreUtil blobStoreUtil = mock(BlobStoreUtil.class);
+    DirDiffUtil dirDiffUtil = mock(DirDiffUtil.class);
+
+    // return immediately without restoring.
+    when(blobStoreUtil.restoreDir(eq(storeDir.toFile()), eq(dirIndex), any(Metadata.class)))
+        .thenReturn(CompletableFuture.completedFuture(null));
+    when(dirDiffUtil.areSameDir(anySet(), anyBoolean())).thenReturn((arg1, arg2) -> true);
+
+    BlobStoreRestoreManager.restoreStores(jobName, jobId, taskName, storesToRestore, prevStoreSnapshotIndexes,
+        loggedBaseDir.toFile(), storageConfig, metrics,
+        storageManagerUtil, blobStoreUtil, dirDiffUtil, EXECUTOR);
+
+    // verify that the store directory restore was called and skipped (i.e. shouldRestore == true)
+    verify(blobStoreUtil, times(1)).restoreDir(eq(storeDir.toFile()), eq(dirIndex), any(Metadata.class));
+    // verify that the store directory was deleted prior to restore
+    // (should still not exist at the end since restore is no-op)
+    assertFalse(storeDir.toFile().exists());
+  }
+
+  @Test
+  public void testRestoreDeletesCheckpointDirsIfRestoring() throws IOException {
+    String jobName = "testJobName";
+    String jobId = "testJobId";
+    TaskName taskName = mock(TaskName.class);
+    BlobStoreRestoreManagerMetrics metrics = new BlobStoreRestoreManagerMetrics(new MetricsRegistryMap());
+    metrics.initStoreMetrics(ImmutableList.of("storeName"));
+    List<String> storesToRestore = ImmutableList.of("storeName");
+    SnapshotIndex snapshotIndex = mock(SnapshotIndex.class);
+    Map<String, Pair<String, SnapshotIndex>> prevStoreSnapshotIndexes =
+        ImmutableMap.of("storeName", Pair.of("blobId", snapshotIndex));
+    DirIndex dirIndex = BlobStoreTestUtil.createDirIndex("[a]");
+    when(snapshotIndex.getDirIndex()).thenReturn(dirIndex);
+    CheckpointId checkpointId = CheckpointId.create();
+    when(snapshotIndex.getSnapshotMetadata())
+        .thenReturn(new SnapshotMetadata(checkpointId, "jobName", "jobId", "taskName", "storeName"));
+
+    Path loggedBaseDir = Files.createTempDirectory(BlobStoreTestUtil.TEMP_DIR_PREFIX);
+
+    // create store dir to be deleted during restore
+    Path storeDir = Files.createTempDirectory(loggedBaseDir, "storeDir");
+    Path storeCheckpointDir1 = Files.createTempDirectory(loggedBaseDir, "storeDir-" + checkpointId);
+    CheckpointId olderCheckpoint = CheckpointId.create();
+    Path storeCheckpointDir2 = Files.createTempDirectory(loggedBaseDir, "storeDir-" + olderCheckpoint);
+    StorageConfig storageConfig = mock(StorageConfig.class);
+    StorageManagerUtil storageManagerUtil = mock(StorageManagerUtil.class);
+    when(storageManagerUtil.getTaskStoreDir(
+        eq(loggedBaseDir.toFile()), eq("storeName"), eq(taskName), eq(TaskMode.Active)))
+        .thenReturn(storeDir.toFile());
+    when(storageManagerUtil.getStoreCheckpointDir(eq(storeDir.toFile()), eq(checkpointId)))
+        .thenReturn(Paths.get(storeDir.toString(), checkpointId.toString()).toString());
+    when(storageManagerUtil.getTaskStoreCheckpointDirs(any(File.class), anyString(), any(TaskName.class), any(TaskMode.class)))
+        .thenReturn(ImmutableList.of(storeCheckpointDir1.toFile(), storeCheckpointDir2.toFile()));
+    BlobStoreUtil blobStoreUtil = mock(BlobStoreUtil.class);
+    DirDiffUtil dirDiffUtil = mock(DirDiffUtil.class);
+
+    when(dirDiffUtil.areSameDir(anySet(), anyBoolean())).thenReturn((arg1, arg2) -> true);
+    // return immediately without restoring.
+    when(blobStoreUtil.restoreDir(eq(storeDir.toFile()), eq(dirIndex), any(Metadata.class)))
+        .thenReturn(CompletableFuture.completedFuture(null));
+
+    BlobStoreRestoreManager.restoreStores(jobName, jobId, taskName, storesToRestore, prevStoreSnapshotIndexes,
+        loggedBaseDir.toFile(), storageConfig, metrics,
+        storageManagerUtil, blobStoreUtil, dirDiffUtil, EXECUTOR);
+
+    // verify that the store directory restore was called and skipped (i.e. shouldRestore == true)
+    verify(blobStoreUtil, times(1)).restoreDir(eq(storeDir.toFile()), eq(dirIndex), any(Metadata.class));
+    // verify that the checkpoint directories were deleted prior to restore (should not exist at the end)
+    assertFalse(storeCheckpointDir1.toFile().exists());
+    assertFalse(storeCheckpointDir2.toFile().exists());
+  }
+
+  @Test
+  public void testRestoreRetainsCheckpointDirsIfValid() throws IOException {
+    String jobName = "testJobName";
+    String jobId = "testJobId";
+    TaskName taskName = mock(TaskName.class);
+    BlobStoreRestoreManagerMetrics metrics = new BlobStoreRestoreManagerMetrics(new MetricsRegistryMap());
+    metrics.initStoreMetrics(ImmutableList.of("storeName"));
+    List<String> storesToRestore = ImmutableList.of("storeName");
+    SnapshotIndex snapshotIndex = mock(SnapshotIndex.class);
+    Map<String, Pair<String, SnapshotIndex>> prevStoreSnapshotIndexes =
+        ImmutableMap.of("storeName", Pair.of("blobId", snapshotIndex));
+    DirIndex dirIndex = BlobStoreTestUtil.createDirIndex("[a]");
+    when(snapshotIndex.getDirIndex()).thenReturn(dirIndex);
+    CheckpointId checkpointId = CheckpointId.create();
+    when(snapshotIndex.getSnapshotMetadata())
+        .thenReturn(new SnapshotMetadata(checkpointId, "jobName", "jobId", "taskName", "storeName"));
+
+    Path loggedBaseDir = Files.createTempDirectory(BlobStoreTestUtil.TEMP_DIR_PREFIX);
+
+    // create store dir to be deleted during restore
+    Path storeDir = Files.createTempDirectory(loggedBaseDir, "storeDir-");
+
+    // create checkpoint dir so that shouldRestore = false (areSameDir == true later)
+    Path storeCheckpointDir = Files.createTempDirectory(loggedBaseDir, "storeDir-" + checkpointId + "-");
+    // create a dummy file to verify after dir rename.
+    Path tempFile = Files.createTempFile(storeCheckpointDir, "tempFile-", null);
+
+    StorageConfig storageConfig = mock(StorageConfig.class);
+    StorageManagerUtil storageManagerUtil = mock(StorageManagerUtil.class);
+    when(storageManagerUtil.getTaskStoreDir(
+        eq(loggedBaseDir.toFile()), eq("storeName"), eq(taskName), eq(TaskMode.Active)))
+        .thenReturn(storeDir.toFile());
+    when(storageManagerUtil.getStoreCheckpointDir(any(File.class), eq(checkpointId)))
+        .thenReturn(storeCheckpointDir.toString());
+    when(storageManagerUtil.getTaskStoreCheckpointDirs(any(File.class), anyString(), any(TaskName.class), any(TaskMode.class)))
+        .thenReturn(ImmutableList.of(storeCheckpointDir.toFile()));
+    BlobStoreUtil blobStoreUtil = mock(BlobStoreUtil.class);
+    DirDiffUtil dirDiffUtil = mock(DirDiffUtil.class);
+
+    // ensures shouldRestore is not called
+    when(dirDiffUtil.areSameDir(anySet(), anyBoolean())).thenReturn((arg1, arg2) -> true);
+    // return immediately without restoring.
+    when(blobStoreUtil.restoreDir(eq(storeDir.toFile()), eq(dirIndex), any(Metadata.class)))
+        .thenReturn(CompletableFuture.completedFuture(null));
+
+    BlobStoreRestoreManager.restoreStores(jobName, jobId, taskName, storesToRestore, prevStoreSnapshotIndexes,
+        loggedBaseDir.toFile(), storageConfig, metrics,
+        storageManagerUtil, blobStoreUtil, dirDiffUtil, EXECUTOR);
+
+    // verify that the store directory restore was not called (should have restored from checkpoint dir)
+    verify(blobStoreUtil, times(0)).restoreDir(eq(storeDir.toFile()), eq(dirIndex), any(Metadata.class));
+    // verify that the checkpoint dir was renamed to store dir
+    assertFalse(storeCheckpointDir.toFile().exists());
+    assertTrue(storeDir.toFile().exists());
+    assertTrue(Files.exists(Paths.get(storeDir.toString(), tempFile.getFileName().toString())));
+  }
+
+  @Test
+  public void testRestoreSkipsStoresWithMissingCheckpointSCM()  {
+    // store renamed from oldStoreName to newStoreName. No SCM for newStoreName in previous checkpoint.
+    String jobName = "testJobName";
+    String jobId = "testJobId";
+    TaskName taskName = mock(TaskName.class);
+    BlobStoreRestoreManagerMetrics metrics = new BlobStoreRestoreManagerMetrics(new MetricsRegistryMap());
+    metrics.initStoreMetrics(ImmutableList.of("newStoreName"));
+    List<String> storesToRestore = ImmutableList.of("newStoreName"); // new store in config
+    SnapshotIndex snapshotIndex = mock(SnapshotIndex.class);
+    Map<String, Pair<String, SnapshotIndex>> prevStoreSnapshotIndexes = mock(Map.class);
+    when(prevStoreSnapshotIndexes.containsKey("newStoreName")).thenReturn(false);
+    DirIndex dirIndex = mock(DirIndex.class);
+    when(snapshotIndex.getDirIndex()).thenReturn(dirIndex);
+    CheckpointId checkpointId = CheckpointId.create();
+    when(snapshotIndex.getSnapshotMetadata())
+        .thenReturn(new SnapshotMetadata(checkpointId, "jobName", "jobId", "taskName", "storeName"));
+    Path loggedBaseDir = mock(Path.class);
+
+    // create store dir to be deleted during restore
+    StorageConfig storageConfig = mock(StorageConfig.class);
+    StorageManagerUtil storageManagerUtil = mock(StorageManagerUtil.class);
+    BlobStoreUtil blobStoreUtil = mock(BlobStoreUtil.class);
+    DirDiffUtil dirDiffUtil = mock(DirDiffUtil.class);
+
+    BlobStoreRestoreManager.restoreStores(jobName, jobId, taskName, storesToRestore, prevStoreSnapshotIndexes,
+        loggedBaseDir.toFile(), storageConfig, metrics,
+        storageManagerUtil, blobStoreUtil, dirDiffUtil, EXECUTOR);
+
+    // verify that we checked the previously checkpointed SCMs.
+    verify(prevStoreSnapshotIndexes, times(1)).containsKey(eq("newStoreName"));
+    // verify that the store directory restore was never called
+    verify(blobStoreUtil, times(0)).restoreDir(any(File.class), any(DirIndex.class), any(Metadata.class));
+  }
+}
diff --git a/samza-core/src/test/java/org/apache/samza/storage/blobstore/serde/TestSnapshotIndexSerde.java b/samza-core/src/test/java/org/apache/samza/storage/blobstore/serde/TestSnapshotIndexSerde.java
new file mode 100644
index 0000000..8103a5b
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/storage/blobstore/serde/TestSnapshotIndexSerde.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.storage.blobstore.serde;
+
+import org.apache.samza.storage.blobstore.index.DirIndex;
+import org.apache.samza.storage.blobstore.index.SnapshotIndex;
+import org.apache.samza.storage.blobstore.index.SnapshotMetadata;
+import org.apache.samza.storage.blobstore.index.serde.SnapshotIndexSerde;
+import org.apache.samza.storage.blobstore.util.BlobStoreTestUtil;
+import java.io.IOException;
+import java.nio.file.Path;
+import java.util.Optional;
+import org.apache.samza.checkpoint.CheckpointId;
+import org.junit.Assert;
+import org.junit.Test;
+
+
+public class TestSnapshotIndexSerde {
+  @Test
+  public void testSnapshotIndexSerde() throws IOException {
+    // create local and remote snapshots
+    String local = "[a, b, c/1, d/1/2]";
+    String remote = "[a, b, z, c/1/2, e/1]";
+
+    Path localSnapshot = BlobStoreTestUtil.createLocalDir(local);
+    DirIndex dirIndex = BlobStoreTestUtil.createDirIndex(remote);
+    SnapshotMetadata snapshotMetadata =
+        new SnapshotMetadata(CheckpointId.create(), "job", "123", "task", "store");
+    SnapshotIndex testRemoteSnapshot =
+        new SnapshotIndex(System.currentTimeMillis(), snapshotMetadata, dirIndex, Optional.empty());
+
+    SnapshotIndexSerde snapshotIndexSerde = new SnapshotIndexSerde();
+    byte[] serialized = snapshotIndexSerde.toBytes(testRemoteSnapshot);
+    SnapshotIndex deserialized = snapshotIndexSerde.fromBytes(serialized);
+
+    Assert.assertNotNull(deserialized);
+    Assert.assertEquals(deserialized, testRemoteSnapshot);
+  }
+}
diff --git a/samza-core/src/test/java/org/apache/samza/storage/blobstore/util/BlobStoreTestUtil.java b/samza-core/src/test/java/org/apache/samza/storage/blobstore/util/BlobStoreTestUtil.java
new file mode 100644
index 0000000..2d72451
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/storage/blobstore/util/BlobStoreTestUtil.java
@@ -0,0 +1,296 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.storage.blobstore.util;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.samza.storage.blobstore.diff.DirDiff;
+import org.apache.samza.storage.blobstore.index.DirIndex;
+import org.apache.samza.storage.blobstore.index.FileBlob;
+import org.apache.samza.storage.blobstore.index.FileIndex;
+import org.apache.samza.storage.blobstore.index.FileMetadata;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.zip.CRC32;
+import java.util.zip.Checksum;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.samza.util.FileUtil;
+
+/**
+ * Test util methods to work with local dirs, {@link DirDiff}s and {@link DirIndex}es.
+ */
+public class BlobStoreTestUtil {
+  public static final String TEMP_DIR_PREFIX = "samza-blob-store-test-";
+
+  public static Path createLocalDir(String files) throws IOException {
+    Path tempDirPath = Files.createTempDirectory(TEMP_DIR_PREFIX);
+    File tempDirFile = tempDirPath.toFile();
+    String tempDirPathString = tempDirPath.toAbsolutePath().toString();
+
+    if (files.length() == 2) return tempDirPath;
+    String[] paths = files.substring(1, files.length() - 1).split(",");
+
+    for (String path: paths) {
+      path = path.trim();
+      if (!path.contains("/")) {
+        Path filePath = Files.createFile(Paths.get(tempDirPathString, path));
+        new FileUtil().writeToTextFile(filePath.toFile(), path, false); // file contents == file name
+        filePath.toFile().deleteOnExit();
+      } else {
+        String dirs = path.substring(0, path.lastIndexOf("/"));
+        String file = path.substring(path.lastIndexOf("/") + 1);
+        Path directories = Files.createDirectories(Paths.get(tempDirPathString, dirs));
+        if (!StringUtils.isBlank(file)) { // can be blank for empty directories
+          Path filePath = Paths.get(directories.toAbsolutePath().toString(), file);
+          Files.createFile(filePath);
+          new FileUtil().writeToTextFile(filePath.toFile(), file, false); // file contents == file name
+          filePath.toFile().deleteOnExit();
+        }
+      }
+    }
+
+    deleteDirRecursivelyOnExit(tempDirFile);
+    return tempDirPath;
+  }
+
+  public static DirIndex createDirIndex(String files) throws IOException {
+    if (files.equals("[]")) { // empty dir
+      return new DirIndex(DirIndex.ROOT_DIR_NAME, Collections.emptyList(), Collections.emptyList(),
+          Collections.emptyList(), Collections.emptyList());
+    }
+
+    String[] paths = files.substring(1, files.length() - 1).split(",");
+    Arrays.sort(paths);
+    // actually create the directory structure in a temp dir so that file properties and checksums can be computed
+    Path localDir = createLocalDir(files);
+    DirTreeNode dirTree = createDirTree(localDir.toAbsolutePath().toString(), paths);
+    return createDirIndex(localDir.toAbsolutePath().toString(), dirTree);
+  }
+
+  public static void getAllAddedInDiff(String basePath, DirDiff dirDiff, Set<String> allAdded) {
+    for (File fileAdded: dirDiff.getFilesAdded()) {
+      allAdded.add(fileAdded.getAbsolutePath().substring(basePath.length() + 1));
+    }
+
+    for (DirDiff dirAdded: dirDiff.getSubDirsAdded()) {
+      getAllAddedInDiff(basePath, dirAdded, allAdded);
+    }
+
+    for (DirDiff dirRetained: dirDiff.getSubDirsRetained()) {
+      getAllAddedInDiff(basePath, dirRetained, allAdded);
+    }
+  }
+
+  public static void getAllRemovedInDiff(String basePath, DirDiff dirDiff, Set<String> allRemoved) {
+    String prefix = basePath.isEmpty() ? basePath : basePath + "/";
+    for (FileIndex fileRemoved: dirDiff.getFilesRemoved()) {
+      allRemoved.add(prefix + fileRemoved.getFileName());
+    }
+
+    for (DirIndex dirRemoved: dirDiff.getSubDirsRemoved()) {
+      getAllRemovedInRemovedSubDir(prefix + dirRemoved.getDirName(), dirRemoved, allRemoved);
+    }
+
+    for (DirDiff dirRetained: dirDiff.getSubDirsRetained()) {
+      getAllRemovedInRetainedSubDir(prefix + dirRetained.getDirName(), dirRetained, allRemoved);
+    }
+  }
+
+  public static void getAllRetainedInDiff(String basePath, DirDiff dirDiff, Set<String> allRetained) {
+    String prefix = basePath.isEmpty() ? basePath : basePath + "/";
+    for (FileIndex fileRetained: dirDiff.getFilesRetained()) {
+      allRetained.add(prefix + fileRetained.getFileName());
+    }
+
+    for (DirDiff dirRetained: dirDiff.getSubDirsRetained()) {
+      getAllRetainedInDiff(prefix + dirRetained.getDirName(), dirRetained, allRetained);
+    }
+  }
+
+  public static void getAllPresentInIndex(String basePath, DirIndex dirIndex, Set<String> allPresent) {
+    String prefix = basePath.isEmpty() ? basePath : basePath + "/";
+
+    for (FileIndex filePresent: dirIndex.getFilesPresent()) {
+      allPresent.add(prefix + filePresent.getFileName());
+    }
+
+    for (DirIndex dirPresent: dirIndex.getSubDirsPresent()) {
+      getAllPresentInIndex(prefix + dirPresent.getDirName(), dirPresent, allPresent);
+    }
+  }
+
+  public static void getAllRemovedInIndex(String basePath, DirIndex dirIndex, Set<String> allRemoved) {
+    String prefix = basePath.isEmpty() ? basePath : basePath + "/";
+
+    for (FileIndex fileRemoved: dirIndex.getFilesRemoved()) {
+      allRemoved.add(prefix + fileRemoved.getFileName());
+    }
+
+    for (DirIndex dirRemoved: dirIndex.getSubDirsRemoved()) {
+      getAllRemovedInRemovedSubDir(prefix + dirRemoved.getDirName(), dirRemoved, allRemoved);
+    }
+
+    for (DirIndex dirPresent: dirIndex.getSubDirsPresent()) {
+      getAllRemovedInIndex(prefix + dirPresent.getDirName(), dirPresent, allRemoved);
+    }
+  }
+
+  public static SortedSet<String> getExpected(String expectedFiles) {
+    if (expectedFiles.length() == 2) return new TreeSet<>();
+    String[] paths = expectedFiles.substring(1, expectedFiles.length() - 1).split(",");
+    SortedSet<String> result = new TreeSet<>();
+    for (String path: paths) {
+      result.add(path.trim());
+    }
+    return result;
+  }
+
+  private static DirIndex createDirIndex(String baseDir, DirTreeNode root) {
+    String dirName = root.fileName;
+    List<FileIndex> filesPresent = new ArrayList<>();
+    List<DirIndex> subDirsPresent = new ArrayList<>();
+
+    List<FileIndex> filesRemoved = Collections.emptyList();
+    List<DirIndex> subDirsRemoved = Collections.emptyList();
+
+    for (DirTreeNode child: root.children) {
+      if (!child.children.isEmpty()) {
+        subDirsPresent.add(createDirIndex(baseDir + "/" + child.fileName, child));
+      } else {
+        filesPresent.add(createFileIndex(baseDir + "/" + child.fileName, child));
+      }
+    }
+
+    return new DirIndex(dirName, filesPresent, filesRemoved, subDirsPresent, subDirsRemoved);
+  }
+
+  private static DirTreeNode createDirTree(String baseDir, String[] paths) {
+    DirTreeNode root = new DirTreeNode();
+    root.fileName = DirIndex.ROOT_DIR_NAME;
+
+    for (String path: paths) {
+      DirTreeNode pathRoot = root;
+      path = path.trim();
+      String[] pathParts = path.split("/");
+      for (String pathPart: pathParts) {
+        DirTreeNode childNode;
+
+        Optional<DirTreeNode> childNodeOptional = pathRoot.children.stream()
+            .filter(dtn -> dtn.fileName.equals(pathPart)).findFirst();
+
+        if (childNodeOptional.isPresent()) {
+          childNode = childNodeOptional.get();
+        } else {
+          childNode = new DirTreeNode();
+          childNode.fileName = pathPart;
+          pathRoot.children.add(childNode);
+        }
+        pathRoot = childNode;
+      }
+    }
+
+    return root;
+  }
+
+  public static File createTestCheckpointDirectory(String storePath, String checkpointId) throws IOException {
+    File checkpointDir = new File(storePath + "-" + checkpointId);
+    FileUtils.copyDirectory(new File(storePath), checkpointDir);
+    return checkpointDir;
+  }
+
+  private static FileIndex createFileIndex(String filePath, DirTreeNode node) {
+    long checksum;
+    FileMetadata fileMetadata;
+    try {
+      Path path = Paths.get(filePath);
+      Checksum crc32 = new CRC32();
+      byte[] fileBytes = Files.readAllBytes(path);
+      crc32.update(fileBytes, 0, fileBytes.length);
+      checksum = crc32.getValue();
+      fileMetadata = FileMetadata.fromFile(path.toFile());
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+    return new FileIndex(node.fileName, ImmutableList.of(new FileBlob(node.fileName, 0)), fileMetadata, checksum);
+  }
+
+  // recursively adds all present files to allRemoved in removed subdirs
+  private static void getAllRemovedInRemovedSubDir(String basePath, DirIndex dirIndex, Set<String> allRemoved) {
+    String prefix = basePath.isEmpty() ? basePath : basePath + "/";
+    for (FileIndex f: dirIndex.getFilesPresent()) {
+      allRemoved.add(prefix + f.getFileName());
+    }
+
+    for (DirIndex d: dirIndex.getSubDirsPresent()) {
+      getAllRemovedInRemovedSubDir(prefix + d.getDirName(), d, allRemoved);
+    }
+  }
+
+  // only adds removed files in retained subdirs
+  private static void getAllRemovedInRetainedSubDir(String basePath, DirDiff dirDiff, Set<String> allRemoved) {
+    String prefix = basePath.isEmpty() ? basePath : basePath + "/";
+    for (FileIndex f: dirDiff.getFilesRemoved()) {
+      allRemoved.add(prefix + f.getFileName());
+    }
+
+    for (DirIndex dirRemoved: dirDiff.getSubDirsRemoved()) {
+      getAllRemovedInRemovedSubDir(prefix + dirRemoved.getDirName(), dirRemoved, allRemoved);
+    }
+
+    for (DirDiff dirRetained: dirDiff.getSubDirsRetained()) {
+      getAllRemovedInRetainedSubDir(prefix + dirRetained.getDirName(), dirRetained, allRemoved);
+    }
+  }
+
+  private static class DirTreeNode {
+    String fileName;
+    Set<DirTreeNode> children = new HashSet<>();
+  }
+
+  private static class MockFileMetadata extends FileMetadata {
+    public MockFileMetadata() {
+      super(0, 0, 0, "owner", "group", "rwxrw-r--");
+    }
+  }
+
+  private static void deleteDirRecursivelyOnExit(File dir) {
+    dir.deleteOnExit();
+    for (File f: dir.listFiles()) {
+      if (f.isDirectory()) {
+        deleteDirRecursivelyOnExit(f);
+      } else {
+        f.deleteOnExit();
+      }
+    }
+  }
+}
diff --git a/samza-core/src/test/java/org/apache/samza/storage/blobstore/util/TestBlobStoreUtil.java b/samza-core/src/test/java/org/apache/samza/storage/blobstore/util/TestBlobStoreUtil.java
new file mode 100644
index 0000000..b22827e
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/storage/blobstore/util/TestBlobStoreUtil.java
@@ -0,0 +1,936 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.storage.blobstore.util;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.util.concurrent.MoreExecutors;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.attribute.FileTime;
+import java.nio.file.attribute.PosixFileAttributes;
+import java.nio.file.attribute.PosixFilePermissions;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.CompletionStage;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.zip.CRC32;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.io.output.NullOutputStream;
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.samza.SamzaException;
+import org.apache.samza.checkpoint.Checkpoint;
+import org.apache.samza.checkpoint.CheckpointId;
+import org.apache.samza.checkpoint.CheckpointV2;
+import org.apache.samza.storage.blobstore.BlobStoreManager;
+import org.apache.samza.storage.blobstore.BlobStoreStateBackendFactory;
+import org.apache.samza.storage.blobstore.Metadata;
+import org.apache.samza.storage.blobstore.diff.DirDiff;
+import org.apache.samza.storage.blobstore.exceptions.DeletedException;
+import org.apache.samza.storage.blobstore.exceptions.RetriableException;
+import org.apache.samza.storage.blobstore.index.DirIndex;
+import org.apache.samza.storage.blobstore.index.FileBlob;
+import org.apache.samza.storage.blobstore.index.FileIndex;
+import org.apache.samza.storage.blobstore.index.FileMetadata;
+import org.apache.samza.storage.blobstore.index.SnapshotIndex;
+import org.apache.samza.storage.blobstore.index.SnapshotMetadata;
+import org.apache.samza.util.FileUtil;
+import org.apache.samza.util.FutureUtil;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+import org.mockito.stubbing.Answer;
+
+import static org.junit.Assert.*;
+import static org.mockito.Matchers.*;
+import static org.mockito.Mockito.*;
+
+
+public class TestBlobStoreUtil {
+  private static final ExecutorService EXECUTOR = MoreExecutors.newDirectExecutorService();
+  private final CheckpointId checkpointId = CheckpointId.deserialize("1234-567");
+  private final String jobName = "jobName";
+  private final String jobId = "jobId";
+  private final String taskName = "taskName";
+  private final String storeName = "storeName";
+  private final Metadata metadata =
+      new Metadata("payload-path", Optional.empty(), jobName, jobId, taskName, storeName);
+
+  @Test
+  // TODO HIGH shesharm test with empty (0 byte) files
+  public void testPutDir() throws IOException, InterruptedException, ExecutionException {
+    BlobStoreManager blobStoreManager = mock(BlobStoreManager.class);
+
+    // File, dir and recursive dir added, retained and removed in local
+    String local = "[a, c, z/1, y/1, p/m/1, q/n/1]";
+    String remote = "[a, b, z/1, x/1, p/m/1, p/m/2, r/o/1]";
+    String expectedAdded = "[c, y/1, q/n/1]";
+    String expectedRetained = "[a, z/1, p/m/1]";
+    String expectedRemoved = "[b, x/1, r/o/1, p/m/2]";
+    SortedSet<String> expectedAddedFiles = BlobStoreTestUtil.getExpected(expectedAdded);
+    SortedSet<String> expectedRetainedFiles = BlobStoreTestUtil.getExpected(expectedRetained);
+    SortedSet<String> expectedPresentFiles = new TreeSet<>(expectedAddedFiles);
+    expectedPresentFiles.addAll(expectedRetainedFiles);
+    SortedSet<String> expectedRemovedFiles = BlobStoreTestUtil.getExpected(expectedRemoved);
+
+    // Set up environment
+    Path localSnapshotDir = BlobStoreTestUtil.createLocalDir(local);
+    String basePath = localSnapshotDir.toAbsolutePath().toString();
+    DirIndex remoteSnapshotDir = BlobStoreTestUtil.createDirIndex(remote);
+    SnapshotMetadata snapshotMetadata = new SnapshotMetadata(checkpointId, jobName, jobId, taskName, storeName);
+    DirDiff dirDiff = DirDiffUtil.getDirDiff(localSnapshotDir.toFile(), remoteSnapshotDir,
+      (localFile, remoteFile) -> localFile.getName().equals(remoteFile.getFileName()));
+
+    SortedSet<String> allUploaded = new TreeSet<>();
+    // Set up mocks
+    when(blobStoreManager.put(any(InputStream.class), any(Metadata.class)))
+        .thenAnswer((Answer<CompletableFuture<String>>) invocation -> {
+          Metadata metadata = invocation.getArgumentAt(1, Metadata.class);
+          String path = metadata.getPayloadPath();
+          allUploaded.add(path.substring(localSnapshotDir.toAbsolutePath().toString().length() + 1));
+          return CompletableFuture.completedFuture(path);
+        });
+
+    // Execute
+    BlobStoreUtil blobStoreUtil = new BlobStoreUtil(blobStoreManager, EXECUTOR, null, null);
+    CompletionStage<DirIndex> dirIndexFuture = blobStoreUtil.putDir(dirDiff, snapshotMetadata);
+    DirIndex dirIndex = null;
+    try {
+      // should be already complete. if not, future composition in putDir is broken.
+      dirIndex = dirIndexFuture.toCompletableFuture().get(0, TimeUnit.MILLISECONDS);
+    } catch (TimeoutException e) {
+      fail("Future returned from putDir should be already complete.");
+    }
+
+    SortedSet<String> allPresent = new TreeSet<>();
+    SortedSet<String> allRemoved = new TreeSet<>();
+    BlobStoreTestUtil.getAllPresentInIndex("", dirIndex, allPresent);
+    BlobStoreTestUtil.getAllRemovedInIndex("", dirIndex, allRemoved);
+
+    // Assert
+    assertEquals(expectedAddedFiles, allUploaded);
+    assertEquals(expectedPresentFiles, allPresent);
+    assertEquals(expectedRemovedFiles, allRemoved);
+  }
+
+  @Test
+  public void testPutDirFailsIfAnyFileUploadFails() throws IOException, TimeoutException, InterruptedException {
+    BlobStoreManager blobStoreManager = mock(BlobStoreManager.class);
+
+    // File, dir and recursive dir added, retained and removed in local
+    String local = "[a, b]";
+    String remote = "[]";
+
+    // Set up environment
+    Path localSnapshotDir = BlobStoreTestUtil.createLocalDir(local);
+    String basePath = localSnapshotDir.toAbsolutePath().toString();
+    DirIndex remoteSnapshotDir = BlobStoreTestUtil.createDirIndex(remote);
+    SnapshotMetadata snapshotMetadata = new SnapshotMetadata(checkpointId, jobName, jobId, taskName, storeName);
+    DirDiff dirDiff = DirDiffUtil.getDirDiff(localSnapshotDir.toFile(), remoteSnapshotDir,
+      (localFile, remoteFile) -> localFile.getName().equals(remoteFile.getFileName()));
+
+    // Set up mocks
+    SamzaException exception = new SamzaException("Error uploading file");
+    CompletableFuture<String> failedFuture = new CompletableFuture<>();
+    failedFuture.completeExceptionally(exception);
+    when(blobStoreManager.put(any(InputStream.class), any(Metadata.class)))
+        .thenAnswer((Answer<CompletableFuture<String>>) invocation -> {
+          Metadata metadata = invocation.getArgumentAt(1, Metadata.class);
+          String path = metadata.getPayloadPath();
+          if (path.endsWith("a")) {
+            return CompletableFuture.completedFuture("aBlobId");
+          } else {
+            return failedFuture;
+          }
+        });
+
+    // Execute
+    BlobStoreUtil blobStoreUtil = new BlobStoreUtil(blobStoreManager, EXECUTOR, null, null);
+    CompletionStage<DirIndex> dirIndexFuture = blobStoreUtil.putDir(dirDiff, snapshotMetadata);
+    try {
+      // should be already complete. if not, future composition in putDir is broken.
+      dirIndexFuture.toCompletableFuture().get(0, TimeUnit.MILLISECONDS);
+    } catch (ExecutionException e) {
+      Throwable cause = e.getCause();
+      // Assert that the result future fails and that the cause is propagated correctly
+      assertEquals(exception, cause);
+      return;
+    }
+
+    fail("DirIndex future should have been completed with an exception");
+  }
+
+  @Test
+  public void testPutDirFailsIfAnySubDirFileUploadFails() throws IOException, TimeoutException, InterruptedException {
+    BlobStoreManager blobStoreManager = mock(BlobStoreManager.class);
+
+    // File, dir and recursive dir added, retained and removed in local
+    String local = "[a/1, b/2]";
+    String remote = "[]";
+
+    // Set up environment
+    Path localSnapshotDir = BlobStoreTestUtil.createLocalDir(local);
+    String basePath = localSnapshotDir.toAbsolutePath().toString();
+    DirIndex remoteSnapshotDir = BlobStoreTestUtil.createDirIndex(remote);
+    SnapshotMetadata snapshotMetadata = new SnapshotMetadata(checkpointId, jobName, jobId, taskName, storeName);
+    DirDiff dirDiff = DirDiffUtil.getDirDiff(localSnapshotDir.toFile(), remoteSnapshotDir,
+      (localFile, remoteFile) -> localFile.getName().equals(remoteFile.getFileName()));
+
+    // Set up mocks
+    SamzaException exception = new SamzaException("Error uploading file");
+    CompletableFuture<String> failedFuture = new CompletableFuture<>();
+    failedFuture.completeExceptionally(exception);
+    when(blobStoreManager.put(any(InputStream.class), any(Metadata.class)))
+        .thenAnswer((Answer<CompletableFuture<String>>) invocation -> {
+          Metadata metadata = invocation.getArgumentAt(1, Metadata.class);
+          String path = metadata.getPayloadPath();
+          if (path.endsWith("1")) {
+            return CompletableFuture.completedFuture("a1BlobId");
+          } else {
+            return failedFuture;
+          }
+        });
+
+    // Execute
+    BlobStoreUtil blobStoreUtil = new BlobStoreUtil(blobStoreManager, EXECUTOR, null, null);
+    CompletionStage<DirIndex> dirIndexFuture = blobStoreUtil.putDir(dirDiff, snapshotMetadata);
+    try {
+      // should be already complete. if not, future composition in putDir is broken.
+      dirIndexFuture.toCompletableFuture().get(0, TimeUnit.MILLISECONDS);
+    } catch (ExecutionException e) {
+      Throwable cause = e.getCause();
+      // Assert that the result future fails and that the cause is propagated correctly
+      assertEquals(exception, cause);
+      return;
+    }
+
+    fail("DirIndex future should have been completed with an exception");
+  }
+
+  @Test
+  public void testCleanup() throws IOException, ExecutionException, InterruptedException {
+    BlobStoreManager blobStoreManager = mock(BlobStoreManager.class);
+
+    // File, dir and recursive dir added, retained and removed in local
+    // Using unique file names since test util uses only the file name (leaf node)
+    // as the mock blob id, not the full file path.
+    String local = "[a, c, z/1, y/2, p/m/3, q/n/4]";
+    String remote = "[a, b, z/1, x/5, p/m/3, r/o/6]";
+    String expectedRemoved = "[b, 5, 6]";
+    // keep only the last character (the file name).
+    SortedSet<String> expectedRemovedFiles = BlobStoreTestUtil.getExpected(expectedRemoved);
+
+    // Set up environment
+    Path localSnapshotDir = BlobStoreTestUtil.createLocalDir(local);
+    String basePath = localSnapshotDir.toAbsolutePath().toString();
+    DirIndex remoteSnapshotDir = BlobStoreTestUtil.createDirIndex(remote);
+    SnapshotMetadata snapshotMetadata = new SnapshotMetadata(checkpointId, jobName, jobId, taskName, storeName);
+    DirDiff dirDiff = DirDiffUtil.getDirDiff(localSnapshotDir.toFile(), remoteSnapshotDir,
+      (localFile, remoteFile) -> localFile.getName().equals(remoteFile.getFileName()));
+
+    BlobStoreUtil blobStoreUtil = new BlobStoreUtil(blobStoreManager, EXECUTOR, null, null);
+    when(blobStoreManager.put(any(InputStream.class), any(Metadata.class)))
+        .thenReturn(CompletableFuture.completedFuture("blobId"));
+    CompletionStage<DirIndex> dirIndexFuture = blobStoreUtil.putDir(dirDiff, snapshotMetadata);
+    DirIndex dirIndex = null;
+    try {
+      // should be already complete. if not, future composition in putDir is broken.
+      dirIndex = dirIndexFuture.toCompletableFuture().get(0, TimeUnit.MILLISECONDS);
+    } catch (TimeoutException e) {
+      fail("Future returned from putDir should be already complete.");
+    }
+
+    // Set up mocks
+    SortedSet<String> allDeleted = new TreeSet<>();
+    when(blobStoreManager.delete(anyString(), any(Metadata.class)))
+        .thenAnswer((Answer<CompletableFuture<Void>>) invocation -> {
+          String blobId = invocation.getArgumentAt(0, String.class);
+          allDeleted.add(blobId);
+          return CompletableFuture.completedFuture(null);
+        });
+
+    // Execute
+    CompletionStage<Void> cleanUpFuture = blobStoreUtil.cleanUpDir(dirIndex, metadata);
+    try {
+      // should be already complete. if not, future composition in putDir is broken.
+      cleanUpFuture.toCompletableFuture().get(0, TimeUnit.MILLISECONDS);
+    } catch (TimeoutException e) {
+      fail("Future returned from putDir should be already complete.");
+    }
+
+    // Assert
+    assertEquals(expectedRemovedFiles, allDeleted);
+  }
+
+  @Test
+  public void testCleanUpFailsIfAnyFileDeleteFails()
+      throws IOException, TimeoutException, InterruptedException, ExecutionException {
+    BlobStoreManager blobStoreManager = mock(BlobStoreManager.class);
+
+    // File, dir and recursive dir added, retained and removed in local
+    // Using unique file names since test util uses only the file name (leaf node)
+    // as the mock blob id, not the full file path.
+    String local = "[a, b]";
+    String remote = "[c, d]";
+
+    // Set up environment
+    Path localSnapshotDir = BlobStoreTestUtil.createLocalDir(local);
+    String basePath = localSnapshotDir.toAbsolutePath().toString();
+    DirIndex remoteSnapshotDir = BlobStoreTestUtil.createDirIndex(remote);
+    SnapshotMetadata snapshotMetadata = new SnapshotMetadata(checkpointId, jobName, jobId, taskName, storeName);
+    DirDiff dirDiff = DirDiffUtil.getDirDiff(localSnapshotDir.toFile(), remoteSnapshotDir,
+      (localFile, remoteFile) -> localFile.getName().equals(remoteFile.getFileName()));
+
+    BlobStoreUtil blobStoreUtil = new BlobStoreUtil(blobStoreManager, EXECUTOR, null, null);
+    when(blobStoreManager.put(any(InputStream.class), any(Metadata.class)))
+        .thenReturn(CompletableFuture.completedFuture("blobId"));
+    CompletionStage<DirIndex> dirIndexFuture = blobStoreUtil.putDir(dirDiff, snapshotMetadata);
+    DirIndex dirIndex = null;
+    try {
+      // should be already complete. if not, future composition in putDir is broken.
+      dirIndex = dirIndexFuture.toCompletableFuture().get(0, TimeUnit.MILLISECONDS);
+    } catch (TimeoutException e) {
+      fail("Future returned from putDir should be already complete.");
+    }
+
+    // Set up mocks
+    SamzaException exception = new SamzaException("Error deleting file");
+    CompletableFuture<Void> failedFuture = new CompletableFuture<>();
+    failedFuture.completeExceptionally(exception);
+    when(blobStoreManager.delete(anyString(), any(Metadata.class)))
+        .thenAnswer((Answer<CompletableFuture<Void>>) invocation -> {
+          String blobId = invocation.getArgumentAt(0, String.class);
+          if (blobId.equals("c")) {
+            return CompletableFuture.completedFuture(null);
+          } else {
+            return failedFuture;
+          }
+        });
+
+    // Execute
+    CompletionStage<Void> cleanUpFuture = blobStoreUtil.cleanUpDir(dirIndex, metadata);
+    try {
+      // should be already complete. if not, future composition in putDir is broken.
+      cleanUpFuture.toCompletableFuture().get(0, TimeUnit.MILLISECONDS);
+    } catch (ExecutionException e) {
+      Throwable cause = e.getCause();
+      // Assert that the result future fails and that the cause is propagated correctly
+      assertEquals(exception, cause);
+      return;
+    }
+
+    fail("Clean up future should have been completed with an exception");
+  }
+
+  @Test
+  public void testCleanUpFailsIfAnySubDirFileDeleteFails()
+      throws IOException, TimeoutException, InterruptedException, ExecutionException {
+    BlobStoreManager blobStoreManager = mock(BlobStoreManager.class);
+
+    // File, dir and recursive dir added, retained and removed in local
+    // Using unique file names since test util uses only the file name (leaf node)
+    // as the mock blob id, not the full file path.
+    String local = "[a/1, b/2]";
+    String remote = "[c/3, d/4]";
+
+    // Set up environment
+    Path localSnapshotDir = BlobStoreTestUtil.createLocalDir(local);
+    String basePath = localSnapshotDir.toAbsolutePath().toString();
+    DirIndex remoteSnapshotDir = BlobStoreTestUtil.createDirIndex(remote);
+    SnapshotMetadata snapshotMetadata = new SnapshotMetadata(checkpointId, jobName, jobId, taskName, storeName);
+    DirDiff dirDiff = DirDiffUtil.getDirDiff(localSnapshotDir.toFile(), remoteSnapshotDir,
+      (localFile, remoteFile) -> localFile.getName().equals(remoteFile.getFileName()));
+
+    BlobStoreUtil blobStoreUtil = new BlobStoreUtil(blobStoreManager, EXECUTOR, null, null);
+    when(blobStoreManager.put(any(InputStream.class), any(Metadata.class)))
+        .thenReturn(CompletableFuture.completedFuture("blobId"));
+    CompletionStage<DirIndex> dirIndexFuture = blobStoreUtil.putDir(dirDiff, snapshotMetadata);
+    DirIndex dirIndex = null;
+    try {
+      // should be already complete. if not, future composition in putDir is broken.
+      dirIndex = dirIndexFuture.toCompletableFuture().get(0, TimeUnit.MILLISECONDS);
+    } catch (TimeoutException e) {
+      fail("Future returned from putDir should be already complete.");
+    }
+
+    // Set up mocks
+    SamzaException exception = new SamzaException("Error deleting file");
+    CompletableFuture<Void> failedFuture = new CompletableFuture<>();
+    failedFuture.completeExceptionally(exception);
+    when(blobStoreManager.delete(anyString(), any(Metadata.class)))
+        .thenAnswer((Answer<CompletableFuture<Void>>) invocation -> {
+          String blobId = invocation.getArgumentAt(0, String.class);
+          if (blobId.equals("3")) { // blob ID == file name (leaf node) in blob store test util
+            return CompletableFuture.completedFuture(null);
+          } else {
+            return failedFuture;
+          }
+        });
+
+    // Execute
+    CompletionStage<Void> cleanUpFuture = blobStoreUtil.cleanUpDir(dirIndex, metadata);
+    try {
+      // should be already complete. if not, future composition in putDir is broken.
+      cleanUpFuture.toCompletableFuture().get(0, TimeUnit.MILLISECONDS);
+    } catch (ExecutionException e) {
+      Throwable cause = e.getCause();
+      // Assert that the result future fails and that the cause is propagated correctly
+      assertEquals(exception, cause);
+      return;
+    }
+
+    fail("Clean up future should have been completed with an exception");
+  }
+
+  @Test
+  public void testRemoveTTL() throws IOException, ExecutionException, InterruptedException {
+    BlobStoreManager blobStoreManager = mock(BlobStoreManager.class);
+
+    // File, dir and recursive dir added, retained and removed in local
+    // Using unique file names since test setup returns it as the blob id
+    String local = "[a, c, z/1, y/2, p/m/3, q/n/4]";
+    String remote = "[a, b, z/1, x/5, p/m/3, r/o/6]";
+    String expectedAdded = "[c, y/2, q/n/4]";
+    String expectedRetained = "[a, z/1, p/m/3]";
+    SortedSet<String> expectedAddedFiles = BlobStoreTestUtil.getExpected(expectedAdded);
+    SortedSet<String> expectedRetainedFiles = BlobStoreTestUtil.getExpected(expectedRetained);
+    SortedSet<String> expectedPresentFiles = new TreeSet<>(expectedAddedFiles);
+    expectedPresentFiles.addAll(expectedRetainedFiles);
+
+    // Set up environment
+    Path localSnapshotDir = BlobStoreTestUtil.createLocalDir(local);
+    String basePath = localSnapshotDir.toAbsolutePath().toString();
+    DirIndex remoteSnapshotDir = BlobStoreTestUtil.createDirIndex(remote);
+    SnapshotMetadata snapshotMetadata = new SnapshotMetadata(checkpointId, jobName, jobId, taskName, storeName);
+    DirDiff dirDiff = DirDiffUtil.getDirDiff(localSnapshotDir.toFile(), remoteSnapshotDir,
+      (localFile, remoteFile) -> localFile.getName().equals(remoteFile.getFileName()));
+
+    when(blobStoreManager.put(any(InputStream.class), any(Metadata.class)))
+        .thenAnswer((Answer<CompletableFuture<String>>) invocation -> {
+          Metadata metadata = invocation.getArgumentAt(1, Metadata.class);
+          String path = metadata.getPayloadPath();
+          String fileName = path.substring(path.length() - 1); // use only the last character as file name
+          return CompletableFuture.completedFuture(fileName);
+        });
+
+    BlobStoreUtil blobStoreUtil = new BlobStoreUtil(blobStoreManager, EXECUTOR, null, null);
+    CompletionStage<DirIndex> dirIndexFuture = blobStoreUtil.putDir(dirDiff, snapshotMetadata);
+    DirIndex dirIndex = null;
+    try {
+      // should be already complete. if not, future composition in putDir is broken.
+      dirIndex = dirIndexFuture.toCompletableFuture().get(0, TimeUnit.MILLISECONDS);
+    } catch (TimeoutException e) {
+      fail("Future returned from putDir should be already complete.");
+    }
+
+    SnapshotIndex mockSnapshotIndex = mock(SnapshotIndex.class);
+    when(mockSnapshotIndex.getSnapshotMetadata()).thenReturn(snapshotMetadata);
+    when(mockSnapshotIndex.getDirIndex()).thenReturn(dirIndex);
+
+    SortedSet<String> allTTLRemoved = new TreeSet<>();
+    when(blobStoreManager.removeTTL(anyString(), any(Metadata.class)))
+        .thenAnswer((Answer<CompletableFuture<String>>) invocation -> {
+          String blobId = invocation.getArgumentAt(0, String.class);
+          allTTLRemoved.add(blobId);
+          return CompletableFuture.completedFuture(null);
+        });
+
+    // Execute
+    blobStoreUtil.removeTTL("snapshotIndexBlobId", mockSnapshotIndex, metadata);
+
+    // Assert
+    SortedSet<String> expectedBlobIds = new TreeSet<>();
+    // test uses unique file name (last char) as the blob ID.
+    expectedPresentFiles.forEach(f -> expectedBlobIds.add(f.substring(f.length() - 1)));
+    expectedBlobIds.add("snapshotIndexBlobId");
+
+    assertEquals(expectedBlobIds, allTTLRemoved);
+  }
+
+  @Test
+  public void testPutFileChecksumAndMetadata() throws IOException, ExecutionException, InterruptedException {
+    // Setup
+    SnapshotMetadata snapshotMetadata = new SnapshotMetadata(checkpointId, jobName, jobId, taskName, storeName);
+    Path path = Files.createTempFile("samza-testPutFileChecksum-", ".tmp");
+    FileUtil fileUtil = new FileUtil();
+    fileUtil.writeToTextFile(path.toFile(), RandomStringUtils.random(1000), false);
+    long expectedChecksum = FileUtils.checksumCRC32(path.toFile());
+
+    BlobStoreManager blobStoreManager = mock(BlobStoreManager.class);
+    ArgumentCaptor<Metadata> argumentCaptor = ArgumentCaptor.forClass(Metadata.class);
+    when(blobStoreManager.put(any(InputStream.class), argumentCaptor.capture())).thenAnswer(
+      (Answer<CompletionStage<String>>) invocation -> {
+        InputStream inputStream = invocation.getArgumentAt(0, InputStream.class);
+        // consume input stream to ensure checksum is calculated
+        IOUtils.copy(inputStream, NullOutputStream.NULL_OUTPUT_STREAM);
+        return CompletableFuture.completedFuture("blobId");
+      });
+
+    BlobStoreUtil blobStoreUtil = new BlobStoreUtil(blobStoreManager, EXECUTOR, null, null);
+
+    CompletionStage<FileIndex> fileIndexFuture = blobStoreUtil.putFile(path.toFile(), snapshotMetadata);
+    FileIndex fileIndex = null;
+    try {
+      // should be already complete. if not, future composition in putFile is broken.
+      fileIndex = fileIndexFuture.toCompletableFuture().get(0, TimeUnit.MILLISECONDS);
+    } catch (TimeoutException e) {
+      fail("Future returned from putFile should be already complete.");
+    }
+
+    // Assert
+    Metadata metadata = (Metadata) argumentCaptor.getValue();
+    assertEquals(path.toAbsolutePath().toString(), metadata.getPayloadPath());
+    assertEquals(path.toFile().length(), Long.valueOf(metadata.getPayloadSize()).longValue());
+    assertEquals(expectedChecksum, fileIndex.getChecksum());
+  }
+
+  @Test
+  public void testAreSameFile() throws IOException {
+    FileUtil fileUtil = new FileUtil();
+    // 1. test with sst file with same attributes
+    Path sstFile = Files.createTempFile("samza-testAreSameFiles-", ".sst");
+
+    PosixFileAttributes sstFileAttribs = Files.readAttributes(sstFile, PosixFileAttributes.class);
+    FileMetadata sstFileMetadata = new FileMetadata(sstFileAttribs.creationTime().toMillis(),
+        sstFileAttribs.lastModifiedTime().toMillis(), sstFileAttribs.size(), sstFileAttribs.owner().toString(),
+        sstFileAttribs.group().toString(), PosixFilePermissions.toString(sstFileAttribs.permissions()));
+    // checksum should be ignored for sst file. Set any dummy value
+    FileIndex sstFileIndex = new FileIndex(sstFile.getFileName().toString(), Collections.emptyList(), sstFileMetadata, 0L);
+
+    assertTrue(DirDiffUtil.areSameFile(false).test(sstFile.toFile(), sstFileIndex));
+
+    // 2. test with sst file with different timestamps
+    // Update last modified time
+    Files.setLastModifiedTime(sstFile, FileTime.fromMillis(System.currentTimeMillis() + 1000L));
+    assertTrue(DirDiffUtil.areSameFile(false).test(sstFile.toFile(), sstFileIndex));
+
+    // 3. test with non-sst files with same metadata and content
+    Path tmpFile = Files.createTempFile("samza-testAreSameFiles-", ".tmp");
+    fileUtil.writeToTextFile(tmpFile.toFile(), RandomStringUtils.random(1000), false);
+
+    PosixFileAttributes tmpFileAttribs = Files.readAttributes(tmpFile, PosixFileAttributes.class);
+    FileMetadata tmpFileMetadata =
+        new FileMetadata(tmpFileAttribs.creationTime().toMillis(), tmpFileAttribs.lastModifiedTime().toMillis(),
+            tmpFileAttribs.size(), tmpFileAttribs.owner().toString(), tmpFileAttribs.group().toString(),
+            PosixFilePermissions.toString(tmpFileAttribs.permissions()));
+    FileIndex tmpFileIndex = new FileIndex(tmpFile.getFileName().toString(), Collections.emptyList(), tmpFileMetadata,
+        FileUtils.checksumCRC32(tmpFile.toFile()));
+
+    assertTrue(DirDiffUtil.areSameFile(false).test(tmpFile.toFile(), tmpFileIndex));
+
+    // 4. test with non-sst files with different attributes
+    // change lastModifiedTime of local file
+    FileTime prevLastModified = tmpFileAttribs.lastModifiedTime();
+    Files.setLastModifiedTime(tmpFile, FileTime.fromMillis(System.currentTimeMillis() + 1000L));
+    assertTrue(DirDiffUtil.areSameFile(false).test(tmpFile.toFile(), tmpFileIndex));
+
+    // change content/checksum of local file
+    Files.setLastModifiedTime(tmpFile, prevLastModified); // reset attributes to match with remote file
+    fileUtil.writeToTextFile(tmpFile.toFile(), RandomStringUtils.random(1000), false); //new content
+    assertFalse(DirDiffUtil.areSameFile(false).test(tmpFile.toFile(), tmpFileIndex));
+  }
+
+  @Test
+  public void testRestoreDirRestoresMultiPartFilesCorrectly() throws IOException {
+    Path restoreDirBasePath = Files.createTempDirectory(BlobStoreTestUtil.TEMP_DIR_PREFIX);
+
+    // remote file == 26 blobs, blob ids from a to z, blob contents from a to z, offsets 0 to 25.
+    DirIndex mockDirIndex = mock(DirIndex.class);
+    when(mockDirIndex.getDirName()).thenReturn(DirIndex.ROOT_DIR_NAME);
+    FileIndex mockFileIndex = mock(FileIndex.class);
+    when(mockFileIndex.getFileName()).thenReturn("1.sst");
+
+    // setup mock file attributes. create a temp file to get current user/group/permissions so that they
+    // match with restored files.
+    File tmpFile = Paths.get(restoreDirBasePath.toString(), "tempfile-" + new Random().nextInt()).toFile();
+    tmpFile.createNewFile();
+    PosixFileAttributes attrs = Files.readAttributes(tmpFile.toPath(), PosixFileAttributes.class);
+    FileMetadata fileMetadata = new FileMetadata(1234L, 1243L, 26, // ctime mtime does not matter. size == 26
+        attrs.owner().getName(), attrs.group().getName(), PosixFilePermissions.toString(attrs.permissions()));
+    when(mockFileIndex.getFileMetadata()).thenReturn(fileMetadata);
+    Files.delete(tmpFile.toPath()); // delete so that it doesn't show up in restored dir contents.
+
+    List<FileBlob> mockFileBlobs = new ArrayList<>();
+    StringBuilder fileContents = new StringBuilder();
+    for (int i = 0; i < 26; i++) {
+      FileBlob mockFileBlob = mock(FileBlob.class);
+      char c = (char) ('a' + i);
+      fileContents.append(c); // blob contents == blobId
+      when(mockFileBlob.getBlobId()).thenReturn(String.valueOf(c));
+      when(mockFileBlob.getOffset()).thenReturn(i);
+      mockFileBlobs.add(mockFileBlob);
+    }
+    when(mockFileIndex.getBlobs()).thenReturn(mockFileBlobs);
+    CRC32 checksum = new CRC32();
+    checksum.update(fileContents.toString().getBytes());
+    when(mockFileIndex.getChecksum()).thenReturn(checksum.getValue());
+    when(mockDirIndex.getFilesPresent()).thenReturn(ImmutableList.of(mockFileIndex));
+
+    BlobStoreManager mockBlobStoreManager = mock(BlobStoreManager.class);
+    when(mockBlobStoreManager.get(anyString(), any(OutputStream.class), any(Metadata.class))).thenAnswer(
+      (Answer<CompletionStage<Void>>) invocationOnMock -> {
+        String blobId = invocationOnMock.getArgumentAt(0, String.class);
+        OutputStream outputStream = invocationOnMock.getArgumentAt(1, OutputStream.class);
+        // blob contents = blob id
+        outputStream.write(blobId.getBytes());
+
+        // force flush so that the checksum calculation later uses the full file contents.
+        ((FileOutputStream) outputStream).getFD().sync();
+        return CompletableFuture.completedFuture(null);
+      });
+
+    BlobStoreUtil blobStoreUtil = new BlobStoreUtil(mockBlobStoreManager, EXECUTOR, null, null);
+    blobStoreUtil.restoreDir(restoreDirBasePath.toFile(), mockDirIndex, metadata).join();
+
+    assertTrue(
+        new DirDiffUtil().areSameDir(Collections.emptySet(), false).test(restoreDirBasePath.toFile(), mockDirIndex));
+  }
+
+  @Test
+  public void testRestoreDirRetriesFileRestoreOnRetriableExceptions() throws IOException {
+    Path restoreDirBasePath = Files.createTempDirectory(BlobStoreTestUtil.TEMP_DIR_PREFIX);
+
+    DirIndex mockDirIndex = mock(DirIndex.class);
+    when(mockDirIndex.getDirName()).thenReturn(DirIndex.ROOT_DIR_NAME);
+    FileIndex mockFileIndex = mock(FileIndex.class);
+    when(mockFileIndex.getFileName()).thenReturn("1.sst");
+
+    // setup mock file attributes. create a temp file to get current user/group/permissions so that they
+    // match with restored files.
+    File tmpFile = Paths.get(restoreDirBasePath.toString(), "tempfile-" + new Random().nextInt()).toFile();
+    tmpFile.createNewFile();
+    byte[] fileContents = "fileContents".getBytes();
+    PosixFileAttributes attrs = Files.readAttributes(tmpFile.toPath(), PosixFileAttributes.class);
+    FileMetadata fileMetadata =
+        new FileMetadata(1234L, 1243L, fileContents.length, // ctime mtime does not matter. size == 26
+            attrs.owner().getName(), attrs.group().getName(), PosixFilePermissions.toString(attrs.permissions()));
+    when(mockFileIndex.getFileMetadata()).thenReturn(fileMetadata);
+    Files.delete(tmpFile.toPath()); // delete so that it doesn't show up in restored dir contents.
+
+    List<FileBlob> mockFileBlobs = new ArrayList<>();
+    FileBlob mockFileBlob = mock(FileBlob.class);
+    when(mockFileBlob.getBlobId()).thenReturn("fileBlobId");
+    when(mockFileBlob.getOffset()).thenReturn(0);
+    mockFileBlobs.add(mockFileBlob);
+    when(mockFileIndex.getBlobs()).thenReturn(mockFileBlobs);
+
+    CRC32 checksum = new CRC32();
+    checksum.update(fileContents);
+    when(mockFileIndex.getChecksum()).thenReturn(checksum.getValue());
+    when(mockDirIndex.getFilesPresent()).thenReturn(ImmutableList.of(mockFileIndex));
+
+    BlobStoreManager mockBlobStoreManager = mock(BlobStoreManager.class);
+    when(mockBlobStoreManager.get(anyString(), any(OutputStream.class), any(Metadata.class))).thenAnswer(
+      (Answer<CompletionStage<Void>>) invocationOnMock -> { // first try, retriable error
+        String blobId = invocationOnMock.getArgumentAt(0, String.class);
+        OutputStream outputStream = invocationOnMock.getArgumentAt(1, OutputStream.class);
+        // write garbage data on first retry to verify that final file contents are correct
+        outputStream.write("bad-data".getBytes());
+        ((FileOutputStream) outputStream).getFD().sync();
+        return FutureUtil.failedFuture(new RetriableException()); // retriable error
+      }).thenAnswer((Answer<CompletionStage<Void>>) invocationOnMock -> { // 2nd try
+        String blobId = invocationOnMock.getArgumentAt(0, String.class);
+        OutputStream outputStream = invocationOnMock.getArgumentAt(1, OutputStream.class);
+        // write correct data on first retry to verify that final file contents are correct
+        outputStream.write(fileContents);
+        ((FileOutputStream) outputStream).getFD().sync();
+        return CompletableFuture.completedFuture(null); // success
+      });
+
+    BlobStoreUtil blobStoreUtil = new BlobStoreUtil(mockBlobStoreManager, EXECUTOR, null, null);
+    blobStoreUtil.restoreDir(restoreDirBasePath.toFile(), mockDirIndex, metadata).join();
+
+    assertTrue(
+        new DirDiffUtil().areSameDir(Collections.emptySet(), false).test(restoreDirBasePath.toFile(), mockDirIndex));
+  }
+
+  @Test
+  public void testRestoreDirFailsRestoreOnNonRetriableExceptions() throws IOException {
+    Path restoreDirBasePath = Files.createTempDirectory(BlobStoreTestUtil.TEMP_DIR_PREFIX);
+
+    DirIndex mockDirIndex = mock(DirIndex.class);
+    when(mockDirIndex.getDirName()).thenReturn(DirIndex.ROOT_DIR_NAME);
+    FileIndex mockFileIndex = mock(FileIndex.class);
+    when(mockFileIndex.getFileName()).thenReturn("1.sst");
+
+    // setup mock file attributes. create a temp file to get current user/group/permissions so that they
+    // match with restored files.
+    File tmpFile = Paths.get(restoreDirBasePath.toString(), "tempfile-" + new Random().nextInt()).toFile();
+    tmpFile.createNewFile();
+    byte[] fileContents = "fileContents".getBytes();
+    PosixFileAttributes attrs = Files.readAttributes(tmpFile.toPath(), PosixFileAttributes.class);
+    FileMetadata fileMetadata =
+        new FileMetadata(1234L, 1243L, fileContents.length, // ctime mtime does not matter. size == 26
+            attrs.owner().getName(), attrs.group().getName(), PosixFilePermissions.toString(attrs.permissions()));
+    when(mockFileIndex.getFileMetadata()).thenReturn(fileMetadata);
+    Files.delete(tmpFile.toPath()); // delete so that it doesn't show up in restored dir contents.
+
+    List<FileBlob> mockFileBlobs = new ArrayList<>();
+    FileBlob mockFileBlob = mock(FileBlob.class);
+    when(mockFileBlob.getBlobId()).thenReturn("fileBlobId");
+    when(mockFileBlob.getOffset()).thenReturn(0);
+    mockFileBlobs.add(mockFileBlob);
+    when(mockFileIndex.getBlobs()).thenReturn(mockFileBlobs);
+
+    CRC32 checksum = new CRC32();
+    checksum.update(fileContents);
+    when(mockFileIndex.getChecksum()).thenReturn(checksum.getValue());
+    when(mockDirIndex.getFilesPresent()).thenReturn(ImmutableList.of(mockFileIndex));
+
+    BlobStoreManager mockBlobStoreManager = mock(BlobStoreManager.class);
+    when(mockBlobStoreManager.get(anyString(), any(OutputStream.class), any(Metadata.class))).thenReturn(
+        FutureUtil.failedFuture(new IllegalArgumentException())) // non retriable error
+        .thenAnswer((Answer<CompletionStage<Void>>) invocationOnMock -> {
+          String blobId = invocationOnMock.getArgumentAt(0, String.class);
+          OutputStream outputStream = invocationOnMock.getArgumentAt(1, OutputStream.class);
+          outputStream.write(fileContents);
+
+          // force flush so that the checksum calculation later uses the full file contents.
+          ((FileOutputStream) outputStream).getFD().sync();
+          return CompletableFuture.completedFuture(null);
+        });
+
+    BlobStoreUtil blobStoreUtil = new BlobStoreUtil(mockBlobStoreManager, EXECUTOR, null, null);
+    try {
+      blobStoreUtil.restoreDir(restoreDirBasePath.toFile(), mockDirIndex, metadata).join();
+      fail("Should have failed on non-retriable errors during file restore");
+    } catch (CompletionException e) {
+      assertTrue(e.getCause() instanceof IllegalArgumentException);
+    }
+  }
+
+  @Test
+  @Ignore // TODO remove
+  public void testRestoreDirRecreatesEmptyFilesAndDirs() throws IOException {
+    String prevSnapshotFiles = "[a, b, z/1, y/1, p/m/1, q/n/1]";
+    DirIndex dirIndex = BlobStoreTestUtil.createDirIndex(prevSnapshotFiles);
+    String localSnapshotFiles = "[a, b, z/1, y/1, p/m/1, q/n/1]";
+    Path localSnapshot = BlobStoreTestUtil.createLocalDir(localSnapshotFiles);
+    BlobStoreManager mockBlobStoreManager = mock(BlobStoreManager.class);
+    when(mockBlobStoreManager.get(anyString(), any(OutputStream.class), any(Metadata.class))).thenAnswer(
+      (Answer<CompletionStage<Void>>) invocationOnMock -> {
+        String blobId = invocationOnMock.getArgumentAt(0, String.class);
+        OutputStream outputStream = invocationOnMock.getArgumentAt(1, OutputStream.class);
+        // blob contents = blob id
+        outputStream.write(blobId.getBytes());
+        return CompletableFuture.completedFuture(null);
+      });
+    boolean result = new DirDiffUtil().areSameDir(new TreeSet<>(), false).test(localSnapshot.toFile(), dirIndex);
+    assertFalse(result);
+    //ToDo complete
+  }
+
+  @Test
+  public void testRestoreDirVerifiesFileChecksums() {
+    // ToDo shesharma restore dir only restores SST files. Since other metadata files are in ignore list,
+    // no checksum matching would be done? Check later.
+  }
+
+  @Test
+  public void testRestoreDirCreatesCorrectDirectoryStructure() throws IOException {
+    String prevSnapshotFiles = "[a, b, z/1, y/1, p/m/1, q/n/1]";
+    DirIndex dirIndex = BlobStoreTestUtil.createDirIndex(prevSnapshotFiles);
+
+    BlobStoreManager mockBlobStoreManager = mock(BlobStoreManager.class);
+    when(mockBlobStoreManager.get(anyString(), any(OutputStream.class), any(Metadata.class))).thenAnswer(
+      (Answer<CompletionStage<Void>>) invocationOnMock -> {
+        String blobId = invocationOnMock.getArgumentAt(0, String.class);
+        OutputStream outputStream = invocationOnMock.getArgumentAt(1, OutputStream.class);
+        // blob contents = blob id
+        outputStream.write(blobId.getBytes());
+        return CompletableFuture.completedFuture(null);
+      });
+
+    Path restoreDirBasePath = Files.createTempDirectory(BlobStoreTestUtil.TEMP_DIR_PREFIX);
+    BlobStoreUtil blobStoreUtil = new BlobStoreUtil(mockBlobStoreManager, EXECUTOR, null, null);
+    blobStoreUtil.restoreDir(restoreDirBasePath.toFile(), dirIndex, metadata).join();
+
+    assertTrue(new DirDiffUtil().areSameDir(Collections.emptySet(), false).test(restoreDirBasePath.toFile(), dirIndex));
+  }
+
+  /**
+   * Tests related to {@link BlobStoreUtil#getStoreSnapshotIndexes}
+   */
+
+  @Test
+  public void testGetSSIReturnsEmptyMapForNullCheckpoint() {
+    BlobStoreUtil blobStoreUtil =
+        new BlobStoreUtil(mock(BlobStoreManager.class), MoreExecutors.newDirectExecutorService(), null, null);
+    Map<String, Pair<String, SnapshotIndex>> snapshotIndexes =
+        blobStoreUtil.getStoreSnapshotIndexes("testJobName", "testJobId", "taskName", null);
+    assertTrue(snapshotIndexes.isEmpty());
+  }
+
+  @Test(expected = SamzaException.class)
+  public void testGetSSIThrowsExceptionForCheckpointV1() {
+    Checkpoint mockCheckpoint = mock(Checkpoint.class);
+    when(mockCheckpoint.getVersion()).thenReturn((short) 1);
+    BlobStoreUtil blobStoreUtil =
+        new BlobStoreUtil(mock(BlobStoreManager.class), MoreExecutors.newDirectExecutorService(), null, null);
+    blobStoreUtil.getStoreSnapshotIndexes("testJobName", "testJobId", "taskName", mockCheckpoint);
+  }
+
+  @Test
+  public void testGetSSIReturnsEmptyMapIfNoEntryForBlobStoreBackendFactory() {
+    CheckpointV2 mockCheckpoint = mock(CheckpointV2.class);
+    when(mockCheckpoint.getVersion()).thenReturn((short) 2);
+    when(mockCheckpoint.getStateCheckpointMarkers()).thenReturn(
+        ImmutableMap.of("com.OtherStateBackendFactory", ImmutableMap.of("storeName", "otherSCM")));
+
+    BlobStoreUtil blobStoreUtil =
+        new BlobStoreUtil(mock(BlobStoreManager.class), MoreExecutors.newDirectExecutorService(), null, null);
+    Map<String, Pair<String, SnapshotIndex>> snapshotIndexes =
+        blobStoreUtil.getStoreSnapshotIndexes("testJobName", "testJobId", "taskName", mockCheckpoint);
+    assertTrue(snapshotIndexes.isEmpty());
+  }
+
+  @Test
+  public void testGetSSIReturnsEmptyMapIfNoStoreForBlobStoreBackendFactory() {
+    CheckpointV2 mockCheckpoint = mock(CheckpointV2.class);
+    when(mockCheckpoint.getVersion()).thenReturn((short) 2);
+    when(mockCheckpoint.getStateCheckpointMarkers()).thenReturn(
+        ImmutableMap.of(BlobStoreStateBackendFactory.class.getName(), ImmutableMap.of()));
+
+    BlobStoreUtil blobStoreUtil =
+        new BlobStoreUtil(mock(BlobStoreManager.class), MoreExecutors.newDirectExecutorService(), null, null);
+    Map<String, Pair<String, SnapshotIndex>> snapshotIndexes =
+        blobStoreUtil.getStoreSnapshotIndexes("testJobName", "testJobId", "taskName", mockCheckpoint);
+    assertTrue(snapshotIndexes.isEmpty());
+  }
+
+  @Test(expected = SamzaException.class)
+  public void testGetSSIThrowsExceptionOnSyncBlobStoreErrors() {
+    Checkpoint checkpoint = createCheckpointV2(BlobStoreStateBackendFactory.class.getName(),
+        ImmutableMap.of("storeName", "snapshotIndexBlobId"));
+    BlobStoreUtil mockBlobStoreUtil = mock(BlobStoreUtil.class);
+    when(mockBlobStoreUtil.getSnapshotIndex(anyString(), any(Metadata.class))).thenThrow(new RuntimeException());
+    when(mockBlobStoreUtil.getStoreSnapshotIndexes(anyString(), anyString(), anyString(),
+        any(Checkpoint.class))).thenCallRealMethod();
+    mockBlobStoreUtil.getStoreSnapshotIndexes("testJobName", "testJobId", "taskName", checkpoint);
+  }
+
+  @Test
+  public void testGetSSISkipsStoresWithSnapshotIndexAlreadyDeleted() {
+    Checkpoint checkpoint = createCheckpointV2(BlobStoreStateBackendFactory.class.getName(),
+        ImmutableMap.of("storeName1", "snapshotIndexBlobId1", "storeName2", "snapshotIndexBlobId2"));
+    SnapshotIndex store1SnapshotIndex = mock(SnapshotIndex.class);
+    BlobStoreUtil mockBlobStoreUtil = mock(BlobStoreUtil.class);
+
+    CompletableFuture<SnapshotIndex> failedFuture = FutureUtil.failedFuture(new DeletedException());
+    when(mockBlobStoreUtil.getSnapshotIndex(eq("snapshotIndexBlobId1"), any(Metadata.class))).thenReturn(
+        CompletableFuture.completedFuture(store1SnapshotIndex));
+    when(mockBlobStoreUtil.getSnapshotIndex(eq("snapshotIndexBlobId2"), any(Metadata.class))).thenReturn(failedFuture);
+    when(mockBlobStoreUtil.getStoreSnapshotIndexes(anyString(), anyString(), anyString(),
+        any(Checkpoint.class))).thenCallRealMethod();
+
+    Map<String, Pair<String, SnapshotIndex>> snapshotIndexes =
+        mockBlobStoreUtil.getStoreSnapshotIndexes("testJobName", "testJobId", "taskName", checkpoint);
+    assertEquals(1, snapshotIndexes.size());
+    assertEquals("snapshotIndexBlobId1", snapshotIndexes.get("storeName1").getLeft());
+    assertEquals(store1SnapshotIndex, snapshotIndexes.get("storeName1").getRight());
+  }
+
+  @Test
+  public void testGetSSIThrowsExceptionIfAnyNonIgnoredAsyncBlobStoreErrors() {
+    Checkpoint checkpoint = createCheckpointV2(BlobStoreStateBackendFactory.class.getName(),
+        ImmutableMap.of("storeName1", "snapshotIndexBlobId1", "storeName2", "snapshotIndexBlobId2"));
+    SnapshotIndex store1SnapshotIndex = mock(SnapshotIndex.class);
+    BlobStoreUtil mockBlobStoreUtil = mock(BlobStoreUtil.class);
+    when(mockBlobStoreUtil.getStoreSnapshotIndexes(anyString(), anyString(), anyString(),
+        any(Checkpoint.class))).thenCallRealMethod();
+    RuntimeException nonIgnoredException = new RuntimeException();
+    CompletableFuture<SnapshotIndex> failedFuture = FutureUtil.failedFuture(nonIgnoredException);
+    when(mockBlobStoreUtil.getSnapshotIndex(eq("snapshotIndexBlobId1"), any(Metadata.class))).thenReturn(
+        FutureUtil.failedFuture(new DeletedException())); // should fail even if some errors are ignored
+    when(mockBlobStoreUtil.getSnapshotIndex(eq("snapshotIndexBlobId2"), any(Metadata.class))).thenReturn(failedFuture);
+
+    try {
+      mockBlobStoreUtil.getStoreSnapshotIndexes("testJobName", "testJobId", "taskName", checkpoint);
+      fail("Should have thrown an exception");
+    } catch (Exception e) {
+      Throwable cause =
+          FutureUtil.unwrapExceptions(CompletionException.class, FutureUtil.unwrapExceptions(SamzaException.class, e));
+      assertEquals(nonIgnoredException, cause);
+    }
+  }
+
+  @Test
+  public void testGetSSIReturnsCorrectSCMSnapshotIndexPair() {
+    String storeName = "storeName";
+    String otherStoreName = "otherStoreName";
+    String storeSnapshotIndexBlobId = "snapshotIndexBlobId";
+    String otherStoreSnapshotIndexBlobId = "otherSnapshotIndexBlobId";
+    SnapshotIndex mockStoreSnapshotIndex = mock(SnapshotIndex.class);
+    SnapshotIndex mockOtherStooreSnapshotIndex = mock(SnapshotIndex.class);
+
+    CheckpointV2 checkpoint = createCheckpointV2(BlobStoreStateBackendFactory.class.getName(),
+        ImmutableMap.of(storeName, storeSnapshotIndexBlobId, otherStoreName, otherStoreSnapshotIndexBlobId));
+
+    BlobStoreUtil mockBlobStoreUtil = mock(BlobStoreUtil.class);
+
+    when(mockBlobStoreUtil.getSnapshotIndex(eq(storeSnapshotIndexBlobId), any(Metadata.class))).thenReturn(
+        CompletableFuture.completedFuture(mockStoreSnapshotIndex));
+    when(mockBlobStoreUtil.getSnapshotIndex(eq(otherStoreSnapshotIndexBlobId), any(Metadata.class))).thenReturn(
+        CompletableFuture.completedFuture(mockOtherStooreSnapshotIndex));
+    when(mockBlobStoreUtil.getStoreSnapshotIndexes(anyString(), anyString(), anyString(),
+        any(Checkpoint.class))).thenCallRealMethod();
+
+    Map<String, Pair<String, SnapshotIndex>> snapshotIndexes =
+        mockBlobStoreUtil.getStoreSnapshotIndexes("testJobName", "testJobId", "taskName", checkpoint);
+
+    assertEquals(storeSnapshotIndexBlobId, snapshotIndexes.get(storeName).getKey());
+    assertEquals(mockStoreSnapshotIndex, snapshotIndexes.get(storeName).getValue());
+    assertEquals(otherStoreSnapshotIndexBlobId, snapshotIndexes.get(otherStoreName).getKey());
+    assertEquals(mockOtherStooreSnapshotIndex, snapshotIndexes.get(otherStoreName).getValue());
+    verify(mockBlobStoreUtil, times(2)).getSnapshotIndex(anyString(), any(Metadata.class));
+  }
+
+  private CheckpointV2 createCheckpointV2(String stateBackendFactory, Map<String, String> storeSnapshotIndexBlobIds) {
+    CheckpointId checkpointId = CheckpointId.create();
+    Map<String, Map<String, String>> factoryStoreSCMs = new HashMap<>();
+    Map<String, String> storeSCMs = new HashMap<>();
+    for (Map.Entry<String, String> entry : storeSnapshotIndexBlobIds.entrySet()) {
+      storeSCMs.put(entry.getKey(), entry.getValue());
+    }
+
+    factoryStoreSCMs.put(stateBackendFactory, storeSCMs);
+    return new CheckpointV2(checkpointId, ImmutableMap.of(), factoryStoreSCMs);
+  }
+}
diff --git a/samza-core/src/test/java/org/apache/samza/storage/blobstore/util/TestDirDiffUtil.java b/samza-core/src/test/java/org/apache/samza/storage/blobstore/util/TestDirDiffUtil.java
new file mode 100644
index 0000000..fa08670
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/storage/blobstore/util/TestDirDiffUtil.java
@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.storage.blobstore.util;
+
+import org.apache.samza.storage.blobstore.diff.DirDiff;
+import org.apache.samza.storage.blobstore.index.DirIndex;
+
+import java.io.IOException;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.junit.Assert.assertEquals;
+
+
+@RunWith(value = Parameterized.class)
+public class TestDirDiffUtil {
+
+  private final String local;
+  private final String remote;
+  private final String expectedAdded;
+  private final String expectedRetained;
+  private final String expectedRemoved;
+  private final String description;
+
+  public TestDirDiffUtil(String local, String remote,
+      String expectedAdded, String expectedRetained, String expectedRemoved,
+      String description) {
+    this.local = local;
+    this.remote = remote;
+    this.expectedAdded = expectedAdded;
+    this.expectedRetained = expectedRetained;
+    this.expectedRemoved = expectedRemoved;
+    this.description = description;
+  }
+
+  // TODO HIGH shesharm test with empty subdirectories
+  @Parameterized.Parameters(name = "testGetDirDiff: {5}")
+  public static Collection<Object[]> data() {
+    return Arrays.asList(new Object[][]{
+        // Local                             Remote                             Expected Added        Expected Retained    Expected Removed     Description
+        // -------------------------------- --------------------------------    -----------------     ------------------   -----------------    ------------------------------------------------------------------
+        {"[]",                               "[]",                               "[]",                "[]",                "[]",                "Nothing in local or remote"},
+        {"[a]",                              "[]",                               "[a]",               "[]",                "[]",                "New file in local"},
+        {"[z/1]",                            "[]",                               "[z/1]",             "[]",                "[]",                "New dir in local"},
+        {"[z/i/1]",                          "[]",                               "[z/i/1]",           "[]",                "[]",                "New recursive dir in local"},
+        {"[a, z/1]",                         "[]",                               "[a, z/1]",          "[]",                "[]",                "New file and dir in local"},
+        {"[a, z/1, y/j/1]",                  "[]",                               "[a, z/1, y/j/1]",   "[]",                "[]",                "New file, dir and recursive dir in local"},
+        {"[a]",                              "[a]",                              "[]",                "[a]",               "[]",                "File retained in local"},
+        {"[z/1]",                            "[z/1]",                            "[]",                "[z/1]",             "[]",                "Dir retained in local"},
+        {"[z/i/1]",                          "[z/i/1]",                          "[]",                "[z/i/1]",           "[]",                "Recursive dir retained in local"},
+        {"[a, z/1]",                         "[a, z/1]",                         "[]",                "[a, z/1]",          "[]",                "File and dir retained in local"},
+        {"[a, z/1, y/j/1]",                  "[a, z/1, y/j/1]",                  "[]",                "[a, z/1, y/j/1]",   "[]",                "File, dir and recursive dir retained in local"},
+        {"[]",                               "[a]",                              "[]",                "[]",                "[a]",               "File removed in local"},
+        {"[]",                               "[z/1]",                            "[]",                "[]",                "[z/1]",             "Dir removed in local"},
+        {"[]",                               "[z/i/1]",                          "[]",                "[]",                "[z/i/1]",           "Recursive dir removed in local"},
+        {"[]",                               "[a, z/1]",                         "[]",                "[]",                "[a, z/1]",          "File and dir removed in local"},
+        {"[]",                               "[a, z/1, y/j/1]",                  "[]",                "[]",                "[a, z/1, y/j/1]",   "File, dir and recursive dir removed in local"},
+        {"[b]",                              "[a]",                              "[b]",               "[]",                "[a]",               "File added and removed in local"},
+        {"[y/1]",                            "[z/1]",                            "[y/1]",             "[]",                "[z/1]",             "Dir added and removed in local"},
+        {"[y/j/1]",                          "[z/i/1]",                          "[y/j/1]",           "[]",                "[z/i/1]",           "Recursive dir added and removed in local"},
+        {"[b, y/1]",                         "[a, z/1]",                         "[b, y/1]",          "[]",                "[a, z/1]",          "File and dir added and removed in local"},
+        {"[b, y/1, x/k/1]",                  "[a, z/1, w/m/1]",                  "[b, y/1, x/k/1]",   "[]",                "[a, z/1, w/m/1]",   "File, dir and recursive dir added and removed in local"},
+        {"[a, c]",                           "[a]",                              "[c]",               "[a]",               "[]",                "File added and retained in local"},
+        {"[z/1, y/1]",                       "[z/1]",                            "[y/1]",             "[z/1]",             "[]",                "Dir added and retained in local"},
+        {"[z/i/1, y/j/1]",                   "[z/i/1]",                          "[y/j/1]",           "[z/i/1]",           "[]",                "Recursive dir added and retained in local"},
+        {"[a, c, z/1, y/1]",                 "[a, z/1]",                         "[c, y/1]",          "[a, z/1]",          "[]",                "File and dir added and retained in local"},
+        {"[a, c, z/1, y/1, p/m/1, q/n/1]",   "[a, z/1, p/m/1]",                  "[c, y/1, q/n/1]",   "[a, z/1, p/m/1]",   "[]",                "File, dir and recursive dir added and retained in local"},
+        {"[a, c]",                           "[a, b]",                           "[c]",               "[a]",               "[b]",               "File added, retained and removed in local"},
+        {"[z/1, y/1]",                       "[z/1, x/1]",                       "[y/1]",             "[z/1]",             "[x/1]",             "Dir added, retained and removed in local"},
+        {"[z/1, z/3]",                       "[z/1, z/2]",                       "[z/3]",             "[z/1]",             "[z/2]",             "File added, retained and removed in dir in local"},
+        {"[z/i/1, y/j/1]",                   "[z/i/1, x/k/1]",                   "[y/j/1]",           "[z/i/1]",           "[x/k/1]",           "Recursive dir added, retained and removed in local"},
+        {"[a, c, z/1, y/1]",                 "[a, b, z/1, x/1]",                 "[c, y/1]",          "[a, z/1]",          "[b, x/1]",          "File and dir added, retained and removed in local"},
+        {"[a, c, z/1, y/1, p/m/1, q/n/1]",   "[a, b, z/1, x/1, p/m/1, r/o/1]",   "[c, y/1, q/n/1]",   "[a, z/1, p/m/1]",   "[b, x/1, r/o/1]",   "File, dir and recursive dir added, retained and removed in local"},
+        {"[a, c, z/1, p/m/1, p/m/2, q/n/1]", "[a, b, z/1, x/1, p/m/1, r/o/1]",   "[c, p/m/2, q/n/1]", "[a, z/1, p/m/1]",   "[b, x/1, r/o/1]",   "File, File in recursive subdir, dir and recursive dir added, retained and removed in local"}
+    });
+  }
+
+  @Test
+  public void testGetDirDiff() throws IOException {
+    // Setup
+    Path localSnapshotDir = BlobStoreTestUtil.createLocalDir(this.local);
+    String basePath = localSnapshotDir.toAbsolutePath().toString();
+    DirIndex remoteSnapshotDir = BlobStoreTestUtil.createDirIndex(this.remote);
+
+    // Execute
+    DirDiff dirDiff = DirDiffUtil.getDirDiff(localSnapshotDir.toFile(), remoteSnapshotDir,
+      (localFile, remoteFile) -> localFile.getName().equals(remoteFile.getFileName()));
+
+    SortedSet<String> allAdded = new TreeSet<>();
+    SortedSet<String> allRemoved = new TreeSet<>();
+    SortedSet<String> allRetained = new TreeSet<>();
+    BlobStoreTestUtil.getAllAddedInDiff(basePath, dirDiff, allAdded);
+    BlobStoreTestUtil.getAllRemovedInDiff("", dirDiff, allRemoved);
+    BlobStoreTestUtil.getAllRetainedInDiff("", dirDiff, allRetained);
+
+    // Assert
+    SortedSet<String> expectedAddedFiles = BlobStoreTestUtil.getExpected(this.expectedAdded);
+    SortedSet<String> expectedRetainedFiles = BlobStoreTestUtil.getExpected(this.expectedRetained);
+    SortedSet<String> expectedRemovedFiles = BlobStoreTestUtil.getExpected(this.expectedRemoved);
+    assertEquals(expectedAddedFiles, allAdded);
+    assertEquals(expectedRetainedFiles, allRetained);
+    assertEquals(expectedRemovedFiles, allRemoved);
+  }
+}
diff --git a/samza-core/src/test/java/org/apache/samza/storage/blobstore/util/TestDirDiffUtilMisc.java b/samza-core/src/test/java/org/apache/samza/storage/blobstore/util/TestDirDiffUtilMisc.java
new file mode 100644
index 0000000..e61297b
--- /dev/null
+++ b/samza-core/src/test/java/org/apache/samza/storage/blobstore/util/TestDirDiffUtilMisc.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.storage.blobstore.util;
+
+import org.apache.samza.storage.blobstore.diff.DirDiff;
+import org.apache.samza.storage.blobstore.index.DirIndex;
+
+import java.io.IOException;
+import java.nio.file.Path;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestDirDiffUtilMisc {
+
+
+  /**
+   * Test the case when a file has been modified locally. I.e., when isSameFile returns false for a local file with
+   * the same name as the remote file. The file should be marked for both deletion and addition.
+   */
+  @Test
+  public void testGetDirDiffWhenIsSameFileReturnsFalseForSameFileName() throws IOException {
+    String local = "[a]";
+    String remote = "[a]";
+    String expectedAdded = "[a]";
+    String expectedRetained = "[]";
+    String expectedRemoved = "[a]";
+
+    Path localSnapshotDir = BlobStoreTestUtil.createLocalDir(local);
+    String basePath = localSnapshotDir.toAbsolutePath().toString();
+    DirIndex remoteSnapshotDir = BlobStoreTestUtil.createDirIndex(remote);
+
+    // Execute
+    DirDiff dirDiff = DirDiffUtil.getDirDiff(localSnapshotDir.toFile(), remoteSnapshotDir,
+      (localFile, remoteFile) -> false);
+
+    SortedSet<String> allAdded = new TreeSet<>();
+    SortedSet<String> allRemoved = new TreeSet<>();
+    SortedSet<String> allRetained = new TreeSet<>();
+    BlobStoreTestUtil.getAllAddedInDiff(basePath, dirDiff, allAdded);
+    BlobStoreTestUtil.getAllRemovedInDiff("", dirDiff, allRemoved);
+    BlobStoreTestUtil.getAllRetainedInDiff("", dirDiff, allRetained);
+
+    // Assert
+    SortedSet<String> expectedAddedFiles = BlobStoreTestUtil.getExpected(expectedAdded);
+    SortedSet<String> expectedRetainedFiles = BlobStoreTestUtil.getExpected(expectedRetained);
+    SortedSet<String> expectedRemovedFiles = BlobStoreTestUtil.getExpected(expectedRemoved);
+    assertEquals(expectedAddedFiles, allAdded);
+    assertEquals(expectedRetainedFiles, allRetained);
+    assertEquals(expectedRemovedFiles, allRemoved);
+  }
+}
diff --git a/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/RocksDbOptionsHelper.java b/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/RocksDbOptionsHelper.java
index b7c47e1..0908a97 100644
--- a/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/RocksDbOptionsHelper.java
+++ b/samza-kv-rocksdb/src/main/java/org/apache/samza/storage/kv/RocksDbOptionsHelper.java
@@ -20,13 +20,17 @@
 package org.apache.samza.storage.kv;
 
 import java.io.File;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.samza.config.Config;
 import org.apache.samza.storage.StorageEngineFactory;
 import org.apache.samza.storage.StorageManagerUtil;
 import org.rocksdb.BlockBasedTableConfig;
+import org.rocksdb.CompactionOptionsUniversal;
+import org.rocksdb.CompactionStopStyle;
 import org.rocksdb.CompactionStyle;
 import org.rocksdb.CompressionType;
 import org.rocksdb.Options;
+import org.rocksdb.WALRecoveryMode;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -36,9 +40,30 @@ import org.slf4j.LoggerFactory;
 public class RocksDbOptionsHelper {
   private static final Logger log = LoggerFactory.getLogger(RocksDbOptionsHelper.class);
 
+  // TODO HIGH pmaheshw Add these to RockdDBTableDescriptor
+  public static final String ROCKSDB_WAL_ENABLED = "rocksdb.wal.enabled";
   private static final String ROCKSDB_COMPRESSION = "rocksdb.compression";
   private static final String ROCKSDB_BLOCK_SIZE_BYTES = "rocksdb.block.size.bytes";
+
+  private static final String ROCKSDB_COMPACTION_NUM_LEVELS = "rocksdb.compaction.num.levels";
+  private static final String ROCKSDB_COMPACTION_LEVEL0_FILE_NUM_COMPACTION_TRIGGER =
+      "rocksdb.compaction.level0.file.num.compaction.trigger";
+  private static final String ROCKSDB_COMPACTION_MAX_BACKGROUND_COMPACTIONS = "rocksdb.compaction.max.background.compactions";
+  private static final String ROCKSDB_COMPACTION_TARGET_FILE_SIZE_BASE = "rocksdb.compaction.target.file.size.base";
+  private static final String ROCKSDB_COMPACTION_TARGET_FILE_SIZE_MULTIPLIER = "rocksdb.compaction.target.file.size.multiplier";
+
   private static final String ROCKSDB_COMPACTION_STYLE = "rocksdb.compaction.style";
+  private static final String ROCKSDB_COMPACTION_UNIVERSAL_MAX_SIZE_AMPLIFICATION_PERCENT =
+      "rocksdb.compaction.universal.max.size.amplification.percent";
+  private static final String ROCKSDB_COMPACTION_UNIVERSAL_SIZE_RATIO =
+      "rocksdb.compaction.universal.size.ratio";
+  private static final String ROCKSDB_COMPACTION_UNIVERSAL_MIN_MERGE_WIDTH =
+      "rocksdb.compaction.universal.min.merge.width";
+  private static final String ROCKSDB_COMPACTION_UNIVERSAL_MAX_MERGE_WIDTH =
+      "rocksdb.compaction.universal.max.merge.width";
+  private static final String ROCKSDB_COMPACTION_UNIVERSAL_COMPACTION_STOP_STYLE =
+      "rocksdb.compaction.universal.compaction.stop.style";
+
   private static final String ROCKSDB_NUM_WRITE_BUFFERS = "rocksdb.num.write.buffers";
   private static final String ROCKSDB_MAX_LOG_FILE_SIZE_BYTES = "rocksdb.max.log.file.size.bytes";
   private static final String ROCKSDB_KEEP_LOG_FILE_NUM = "rocksdb.keep.log.file.num";
@@ -47,6 +72,12 @@ public class RocksDbOptionsHelper {
 
   public static Options options(Config storeConfig, int numTasksForContainer, File storeDir, StorageEngineFactory.StoreMode storeMode) {
     Options options = new Options();
+
+    if (storeConfig.getBoolean(ROCKSDB_WAL_ENABLED, false)) {
+      options.setManualWalFlush(true); // store.flush() will flushWAL(sync = true) instead
+      options.setWalRecoveryMode(WALRecoveryMode.AbsoluteConsistency);
+    }
+
     Long writeBufSize = storeConfig.getLong("container.write.buffer.size.bytes", 32 * 1024 * 1024);
     // Cache size and write buffer size are specified on a per-container basis.
     options.setWriteBufferSize((int) (writeBufSize / numTasksForContainer));
@@ -84,23 +115,7 @@ public class RocksDbOptionsHelper {
     tableOptions.setBlockCacheSize(blockCacheSize).setBlockSize(blockSize);
     options.setTableFormatConfig(tableOptions);
 
-    CompactionStyle compactionStyle = CompactionStyle.UNIVERSAL;
-    String compactionStyleInConfig = storeConfig.get(ROCKSDB_COMPACTION_STYLE, "universal");
-    switch (compactionStyleInConfig) {
-      case "universal":
-        compactionStyle = CompactionStyle.UNIVERSAL;
-        break;
-      case "fifo":
-        compactionStyle = CompactionStyle.FIFO;
-        break;
-      case "level":
-        compactionStyle = CompactionStyle.LEVEL;
-        break;
-      default:
-        log.warn("Unknown rocksdb.compaction.style " + compactionStyleInConfig +
-            ", overwriting to " + compactionStyle.name());
-    }
-    options.setCompactionStyle(compactionStyle);
+    setCompactionOptions(storeConfig, options);
 
     options.setMaxWriteBufferNumber(storeConfig.getInt(ROCKSDB_NUM_WRITE_BUFFERS, 3));
     options.setCreateIfMissing(true);
@@ -125,6 +140,81 @@ public class RocksDbOptionsHelper {
     return options;
   }
 
+  private static void setCompactionOptions(Config storeConfig, Options options) {
+    if (storeConfig.containsKey(ROCKSDB_COMPACTION_NUM_LEVELS)) {
+      options.setNumLevels(storeConfig.getInt(ROCKSDB_COMPACTION_NUM_LEVELS));
+    }
+
+    if (storeConfig.containsKey(ROCKSDB_COMPACTION_LEVEL0_FILE_NUM_COMPACTION_TRIGGER)) {
+      int level0FileNumCompactionTrigger = storeConfig.getInt(ROCKSDB_COMPACTION_LEVEL0_FILE_NUM_COMPACTION_TRIGGER);
+      options.setLevel0FileNumCompactionTrigger(level0FileNumCompactionTrigger);
+    }
+
+    if (storeConfig.containsKey(ROCKSDB_COMPACTION_MAX_BACKGROUND_COMPACTIONS)) {
+      options.setMaxBackgroundCompactions(storeConfig.getInt(ROCKSDB_COMPACTION_MAX_BACKGROUND_COMPACTIONS));
+    }
+
+    if (storeConfig.containsKey(ROCKSDB_COMPACTION_TARGET_FILE_SIZE_BASE)) {
+      options.setTargetFileSizeBase(storeConfig.getLong(ROCKSDB_COMPACTION_TARGET_FILE_SIZE_BASE));
+    }
+
+    if (storeConfig.containsKey(ROCKSDB_COMPACTION_TARGET_FILE_SIZE_MULTIPLIER)) {
+      options.setTargetFileSizeBase(storeConfig.getLong(ROCKSDB_COMPACTION_TARGET_FILE_SIZE_MULTIPLIER));
+    }
+
+    CompactionStyle compactionStyle = CompactionStyle.UNIVERSAL;
+    String compactionStyleInConfig = storeConfig.get(ROCKSDB_COMPACTION_STYLE, "universal");
+    switch (compactionStyleInConfig) {
+      case "universal":
+        compactionStyle = CompactionStyle.UNIVERSAL;
+        break;
+      case "fifo":
+        compactionStyle = CompactionStyle.FIFO;
+        break;
+      case "level":
+        compactionStyle = CompactionStyle.LEVEL;
+        break;
+      default:
+        log.warn("Unknown rocksdb.compaction.style " + compactionStyleInConfig +
+            ", overwriting to " + compactionStyle.name());
+    }
+    options.setCompactionStyle(compactionStyle);
+
+    // Universal compaction options
+    if (compactionStyle.equals(CompactionStyle.UNIVERSAL)) {
+      CompactionOptionsUniversal compactionOptions = new CompactionOptionsUniversal();
+
+      if (storeConfig.containsKey(ROCKSDB_COMPACTION_UNIVERSAL_MAX_SIZE_AMPLIFICATION_PERCENT)) {
+        int val = storeConfig.getInt(ROCKSDB_COMPACTION_UNIVERSAL_MAX_SIZE_AMPLIFICATION_PERCENT);
+        compactionOptions.setMaxSizeAmplificationPercent(val);
+      }
+
+      if (storeConfig.containsKey(ROCKSDB_COMPACTION_UNIVERSAL_SIZE_RATIO)) {
+        int val = storeConfig.getInt(ROCKSDB_COMPACTION_UNIVERSAL_SIZE_RATIO);
+        compactionOptions.setSizeRatio(val);
+      }
+
+      if (storeConfig.containsKey(ROCKSDB_COMPACTION_UNIVERSAL_MIN_MERGE_WIDTH)) {
+        int val = storeConfig.getInt(ROCKSDB_COMPACTION_UNIVERSAL_MIN_MERGE_WIDTH);
+        compactionOptions.setMinMergeWidth(val);
+      }
+
+      if (storeConfig.containsKey(ROCKSDB_COMPACTION_UNIVERSAL_MAX_MERGE_WIDTH)) {
+        int val = storeConfig.getInt(ROCKSDB_COMPACTION_UNIVERSAL_MAX_MERGE_WIDTH);
+        compactionOptions.setMaxMergeWidth(val);
+      }
+
+      if (storeConfig.containsKey(ROCKSDB_COMPACTION_UNIVERSAL_COMPACTION_STOP_STYLE)) {
+        String stopStyle = storeConfig.get(ROCKSDB_COMPACTION_UNIVERSAL_COMPACTION_STOP_STYLE);
+        if (StringUtils.isNotBlank(stopStyle)) {
+          compactionOptions.setStopStyle(CompactionStopStyle.valueOf(stopStyle));
+        }
+      }
+
+      options.setCompactionOptionsUniversal(compactionOptions);
+    }
+  }
+
   public static Long getBlockCacheSize(Config storeConfig, int numTasksForContainer) {
     long cacheSize = storeConfig.getLong("container.cache.size.bytes", 100 * 1024 * 1024L);
     return cacheSize / numTasksForContainer;
diff --git a/samza-kv-rocksdb/src/main/scala/org/apache/samza/storage/kv/RocksDbKeyValueStorageEngineFactory.scala b/samza-kv-rocksdb/src/main/scala/org/apache/samza/storage/kv/RocksDbKeyValueStorageEngineFactory.scala
index e1f44d2..afc289a 100644
--- a/samza-kv-rocksdb/src/main/scala/org/apache/samza/storage/kv/RocksDbKeyValueStorageEngineFactory.scala
+++ b/samza-kv-rocksdb/src/main/scala/org/apache/samza/storage/kv/RocksDbKeyValueStorageEngineFactory.scala
@@ -50,7 +50,13 @@ class RocksDbKeyValueStorageEngineFactory [K, V] extends BaseKeyValueStorageEngi
       () => RocksDbOptionsHelper.getBlockCacheSize(storageConfigSubset, numTasksForContainer))
 
     val rocksDbOptions = RocksDbOptionsHelper.options(storageConfigSubset, numTasksForContainer, storeDir, storeMode)
-    val rocksDbWriteOptions = new WriteOptions().setDisableWAL(true)
+    val rocksDbWriteOptions = new WriteOptions()
+
+    if (!storageConfigSubset.getBoolean(RocksDbOptionsHelper.ROCKSDB_WAL_ENABLED, false)) {
+      // if WAL not enabled, explicitly disable it
+      rocksDbWriteOptions.setDisableWAL(true)
+    }
+
     val rocksDbFlushOptions = new FlushOptions().setWaitForFlush(true)
     val rocksDb = new RocksDbKeyValueStore(
       storeDir,
diff --git a/samza-kv-rocksdb/src/main/scala/org/apache/samza/storage/kv/RocksDbKeyValueStore.scala b/samza-kv-rocksdb/src/main/scala/org/apache/samza/storage/kv/RocksDbKeyValueStore.scala
index 9d5ddfd..6f5a7f2 100644
--- a/samza-kv-rocksdb/src/main/scala/org/apache/samza/storage/kv/RocksDbKeyValueStore.scala
+++ b/samza-kv-rocksdb/src/main/scala/org/apache/samza/storage/kv/RocksDbKeyValueStore.scala
@@ -19,6 +19,8 @@
 
 package org.apache.samza.storage.kv
 
+import com.google.common.annotations.VisibleForTesting
+
 import java.io.File
 import java.nio.file.{Path, Paths}
 import java.util.concurrent.TimeUnit
@@ -125,7 +127,7 @@ class RocksDbKeyValueStore(
 
   // lazy val here is important because the store directories do not exist yet, it can only be opened
   // after the directories are created, which happens much later from now.
-  private lazy val db = RocksDbKeyValueStore.openDB(dir, options, storeConfig, isLoggedStore, storeName, metrics)
+  @VisibleForTesting lazy val db = RocksDbKeyValueStore.openDB(dir, options, storeConfig, isLoggedStore, storeName, metrics)
   private val lexicographic = new LexicographicComparator()
 
   /**
@@ -236,13 +238,17 @@ class RocksDbKeyValueStore(
   def flush(): Unit = ifOpen {
     metrics.flushes.inc
     trace("Flushing store: %s" format storeName)
-    db.flush(flushOptions)
+    if (storeConfig.getBoolean(RocksDbOptionsHelper.ROCKSDB_WAL_ENABLED, false)) {
+      db.flushWal(true)
+    } else {
+      db.flush(flushOptions)
+    }
     trace("Flushed store: %s" format storeName)
   }
 
   override def checkpoint(id: CheckpointId): Optional[Path] = {
     val checkpoint = Checkpoint.create(db)
-    val checkpointPath = StorageManagerUtil.getCheckpointDirPath(dir, id)
+    val checkpointPath = new StorageManagerUtil().getStoreCheckpointDir(dir, id)
     checkpoint.createCheckpoint(checkpointPath)
     Optional.of(Paths.get(checkpointPath))
   }
diff --git a/samza-kv/src/main/java/org/apache/samza/storage/kv/BaseKeyValueStorageEngineFactory.java b/samza-kv/src/main/java/org/apache/samza/storage/kv/BaseKeyValueStorageEngineFactory.java
index 704e0cb..d5059b3 100644
--- a/samza-kv/src/main/java/org/apache/samza/storage/kv/BaseKeyValueStorageEngineFactory.java
+++ b/samza-kv/src/main/java/org/apache/samza/storage/kv/BaseKeyValueStorageEngineFactory.java
@@ -106,7 +106,7 @@ public abstract class BaseKeyValueStorageEngineFactory<K, V> implements StorageE
       storePropertiesBuilder.setPersistedToDisk(true);
     }
     // The store is durable iff it is backed by the task backup manager
-    List<String> storeBackupManager = storageConfig.getStoreBackupManagerClassName(storeName);
+    List<String> storeBackupManager = storageConfig.getStoreBackupFactory(storeName);
     storePropertiesBuilder.setIsDurable(!storeBackupManager.isEmpty());
 
     int batchSize = storageConfigSubset.getInt(WRITE_BATCH_SIZE, DEFAULT_WRITE_BATCH_SIZE);
diff --git a/samza-kv/src/test/java/org/apache/samza/storage/kv/TestBaseKeyValueStorageEngineFactory.java b/samza-kv/src/test/java/org/apache/samza/storage/kv/TestBaseKeyValueStorageEngineFactory.java
index 05b106d..457fcba 100644
--- a/samza-kv/src/test/java/org/apache/samza/storage/kv/TestBaseKeyValueStorageEngineFactory.java
+++ b/samza-kv/src/test/java/org/apache/samza/storage/kv/TestBaseKeyValueStorageEngineFactory.java
@@ -139,7 +139,7 @@ public class TestBaseKeyValueStorageEngineFactory {
   @Test
   public void testDurableKeyValueStore() {
     Config config = new MapConfig(BASE_CONFIG, DISABLE_CACHE,
-        ImmutableMap.of(String.format(StorageConfig.STORE_BACKEND_BACKUP_FACTORIES, STORE_NAME),
+        ImmutableMap.of(String.format(StorageConfig.STORE_BACKUP_FACTORIES, STORE_NAME),
         "backendFactory,backendFactory2"));
     StorageEngine storageEngine = callGetStorageEngine(config, null);
     KeyValueStorageEngine<?, ?> keyValueStorageEngine = baseStorageEngineValidation(storageEngine);