You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@samza.apache.org by "shekhars-li (via GitHub)" <gi...@apache.org> on 2023/06/24 01:16:37 UTC

[GitHub] [samza] shekhars-li opened a new pull request, #1676: GetDeleted API and Recover from DeletedException

shekhars-li opened a new pull request, #1676:
URL: https://github.com/apache/samza/pull/1676

   [WIP - Do not merge yet]
   Problem Statement:
   - Yarn can sometimes create orphaned containers. In our production systems, we noticed that there were overlapping Samza containers running/committing at the same time. 
   - If the stores are backed up to a blob store, this orphaned and overlapping container may delete a blob (which is common during delta state calculation in commit lifecycle with blob store backend). The other non-orphaned container may expect this blob to be present. 
   - This causes the container and subsequently the job to fail. During this, the container fails with DeletedException - which is Blob store's response that the blob was present but is gone now. 
   
   Fix:
   - During commit, if a container fails with DeletedException, let it fail/restart.
   - During the recovery phase of the restart, get the deleted blob with get() call with getDeleted flag that indicates that if the blob is tombstoned but not compacted, blob store will return it. 
   - Recreate the new blob by uploading it to blob store afresh. Use the new blob id received to create a new checkpoint. 
   - After this, and as long as orphaned container is not cleaned up by Yarn, the container should be able to commit regulary. 
   
   Tests:
   TBD


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] shekhars-li commented on a diff in pull request #1676: [SAMZA-2787] GetDeleted API and Recover from DeletedException

Posted by "shekhars-li (via GitHub)" <gi...@apache.org>.
shekhars-li commented on code in PR #1676:
URL: https://github.com/apache/samza/pull/1676#discussion_r1275578391


##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManagerUtil.java:
##########
@@ -411,4 +438,292 @@ public static Set<String> getSideInputStoreNames(
     }
     return sideInputStores;
   }
+
+  public static List<Future<Void>> initAndRestoreTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, SamzaContainerMetrics samzaContainerMetrics,
+      CheckpointManager checkpointManager, JobContext jobContext, ContainerModel containerModel,
+      Map<TaskName, Checkpoint> taskCheckpoints, Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames,
+      Config config, ExecutorService executor, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      File loggerStoreDir, Map<String, SystemConsumer> storeConsumers) {
+
+    Set<String> forceRestoreTasks = new HashSet<>();
+    // Initialize each TaskStorageManager.
+    taskRestoreManagers.forEach((taskName, restoreManagers) ->
+        restoreManagers.forEach((factoryName, taskRestoreManager) -> {
+          try {
+            taskRestoreManager.init(taskCheckpoints.get(taskName));
+          } catch (SamzaException ex) {
+            if (isUnwrappedExceptionDeletedException(ex) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+              // Get deleted SnapshotIndex blob with GetDeleted and mark the task to be restored with GetDeleted as well.
+              // this ensures that the restore downloads the snapshot, recreates a new snapshot, uploads it to blob store
+              // and creates a new checkpoint.
+              ((BlobStoreRestoreManager) taskRestoreManager).init(taskCheckpoints.get(taskName), true);
+              forceRestoreTasks.add(taskName.getTaskName());
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("init failed for task: %s with GetDeleted set to true", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex);
+            }
+          }
+        })
+    );
+
+    // Start each store consumer once.
+    // Note: These consumers are per system and only changelog system store consumers will be started.
+    // Some TaskRestoreManagers may not require the consumer to be started, but due to the agnostic nature of
+    // ContainerStorageManager we always start the changelog consumer here in case it is required
+    storeConsumers.values().stream().distinct().forEach(SystemConsumer::start);
+
+    return restoreAllTaskInstances(taskRestoreManagers, taskCheckpoints, taskBackendFactoryToStoreNames, jobContext,
+        containerModel, samzaContainerMetrics, checkpointManager, config, taskInstanceMetrics, executor, loggerStoreDir,
+        forceRestoreTasks);
+  }
+
+  /**
+   * Restores all TaskInstances and returns a future for each TaskInstance restore. If this restore fails with DeletedException
+   * it will retry the restore with getDeleted flag, get all the blobs, recreate a new checkpoint in blob store and
+   * write that checkpoint to checkpoint topic.
+   * @param taskRestoreManagers map of TaskName to factory name to TaskRestoreManager map.
+   */
+  public static List<Future<Void>> restoreAllTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, Map<TaskName, Checkpoint> taskCheckpoints,
+      Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames, JobContext jobContext,
+      ContainerModel containerModel, SamzaContainerMetrics samzaContainerMetrics, CheckpointManager checkpointManager,
+      Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics, ExecutorService executor,
+      File loggedStoreDir, Set<String> forceRestoreTask) {
+
+    List<Future<Void>> taskRestoreFutures = new ArrayList<>();
+
+    // Submit restore callable for each taskInstance
+    taskRestoreManagers.forEach((taskInstanceName, restoreManagersMap) -> {
+      // Submit for each restore factory
+      restoreManagersMap.forEach((factoryName, taskRestoreManager) -> {
+        long startTime = System.currentTimeMillis();
+        String taskName = taskInstanceName.getTaskName();
+        LOG.info("Starting restore for state for task: {}", taskName);
+
+        CompletableFuture<Void> restoreFuture;
+        if (forceRestoreTask.contains(taskName) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+          restoreFuture = ((BlobStoreRestoreManager) taskRestoreManager).restore(true);
+        } else {
+          restoreFuture = taskRestoreManager.restore();
+        }
+
+        CompletableFuture<Void> taskRestoreFuture = restoreFuture.handle((res, ex) -> {
+          updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+
+          if (ex != null) {
+            if (isUnwrappedExceptionDeletedException(ex)) {
+              LOG.warn(
+                  "Restore state for task: {} received DeletedException. Reattempting with getDeletedBlobs enabled",
+                  taskInstanceName.getTaskName());
+
+              // Try to restore with getDeleted flag
+              CompletableFuture<Void> future =
+                  restoreTaskStoresAndCreateCheckpointWithGetDeleted(taskInstanceName, taskCheckpoints, checkpointManager,
+                      taskRestoreManager, config, taskInstanceMetrics, executor,
+                      taskBackendFactoryToStoreNames.get(taskInstanceName).get(factoryName), loggedStoreDir, jobContext,
+                      containerModel);
+              try {
+                if (future != null) {
+                  future.join(); // Block and restore deleted state before continuing
+                }
+              } catch (Exception e) {
+                String msg = String.format("Unable to recover from DeletedException for task %s.", taskName);
+                throw new SamzaException(msg, e);
+              } finally {
+                updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+              }
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("Error restoring state for task: %s", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex); // wrap in unchecked exception to throw from lambda
+            }
+          }
+
+          // Stop all persistent stores after restoring. Certain persistent stores opened in BulkLoad mode are compacted
+          // on stop, so paralleling stop() also parallelizes their compaction (a time-intensive operation).
+          try {
+            taskRestoreManager.close();
+          } catch (Exception e) {
+            LOG.error("Error closing restore manager for task: {} after {} restore", taskName,
+                ex != null ? "unsuccessful" : "successful", e);
+            // ignore exception from close. container may still be able to continue processing/backups
+            // if restore manager close fails.
+          }
+          return null;
+        });
+        taskRestoreFutures.add(taskRestoreFuture);
+      });
+    });
+
+    return taskRestoreFutures;
+  }
+
+  public static CompletableFuture<Void> restoreTaskStoresAndCreateCheckpointWithGetDeleted(TaskName taskName,
+      Map<TaskName, Checkpoint> taskCheckpoints, CheckpointManager checkpointManager,
+      TaskRestoreManager taskRestoreManager, Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      ExecutorService executor, Set<String> storesToRestore, File loggedStoreBaseDirectory, JobContext jobContext,
+      ContainerModel containerModel) {
+
+    // if taskInstanceMetrics are specified use those for store metrics,
+    // otherwise (in case of StorageRecovery) use a blank MetricsRegistryMap
+    MetricsRegistry metricsRegistry =
+        taskInstanceMetrics.get(taskName) != null ? taskInstanceMetrics.get(taskName).registry()
+            : new MetricsRegistryMap();
+
+    BlobStoreManager blobStoreManager = getBlobStoreManager(config, executor);
+    JobConfig jobConfig = new JobConfig(config);
+    BlobStoreUtil blobStoreUtil =
+        new BlobStoreUtil(blobStoreManager, executor, new BlobStoreConfig(config), null,
+            new BlobStoreRestoreManagerMetrics(metricsRegistry));
+
+    BlobStoreRestoreManager blobStoreRestoreManager = (BlobStoreRestoreManager) taskRestoreManager;
+
+    CheckpointId checkpointId = CheckpointId.create();
+    Map<String, String> oldSCMs = ((CheckpointV2) taskCheckpoints.get(taskName))
+        .getStateCheckpointMarkers().get(BlobStoreStateBackendFactory.class.getName());
+
+    // Returns a single future that guarantees all the following are completed, in this order:
+    // 1. Restore state locally by getting deleted blobs from the blob store.
+    // 2. Create a new Checkpoint from restored state and back it up on the blob store.
+    // 3. Clean up old/deleted Snapshot
+    // 4. Remove TTL from the new Snapshot on the blob store
+    // 5. Write the new checkpoint to checkpoint topic and update taskCheckpoints map
+
+    // 1. restore state with getDeleted flag set to true
+    return blobStoreRestoreManager.restore(true).thenCompose(r -> {
+      // 2. Create and backup new checkpoint on the blob store.
+      CompletableFuture<Map<String, String>> uploadSCMs =
+          createNewCheckpointAndBackupStores(jobContext, containerModel, config, taskName, storesToRestore, checkpointId,
+              loggedStoreBaseDirectory, blobStoreManager, metricsRegistry, executor);
+      CompletableFuture<Void> future = uploadSCMs.thenCompose(scms -> {
+        // 3. Delete prev SnapshotIndex including files/subdirs,
+        Map<String, CompletionStage<Void>> deleteOldSnapshotsFutures =
+            deletePrevSnapshots(blobStoreUtil, oldSCMs, taskName, jobConfig);
+        // 4. Mark new Snapshots to never expire

Review Comment:
   Yes. This method removes TTL of all the new blobs and then the SnapshotIndex blob. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] prateekm commented on a diff in pull request #1676: [SAMZA-2787] GetDeleted API and Recover from DeletedException

Posted by "prateekm (via GitHub)" <gi...@apache.org>.
prateekm commented on code in PR #1676:
URL: https://github.com/apache/samza/pull/1676#discussion_r1287582032


##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/util/BlobStoreUtil.java:
##########
@@ -114,6 +114,11 @@ public BlobStoreUtil(BlobStoreManager blobStoreManager, ExecutorService executor
    */
   public Map<String, Pair<String, SnapshotIndex>> getStoreSnapshotIndexes(
       String jobName, String jobId, String taskName, Checkpoint checkpoint, Set<String> storesToBackupOrRestore) {
+    return getStoreSnapshotIndexes(jobName, jobId, taskName, checkpoint, storesToBackupOrRestore, false);
+  }
+
+  public Map<String, Pair<String, SnapshotIndex>> getStoreSnapshotIndexes(String jobName, String jobId, String taskName,

Review Comment:
   See comments in BSRM and BSBM init. When do we need to get snapshot indexes with getDeletedBlobs = false?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] prateekm commented on a diff in pull request #1676: [SAMZA-2787] GetDeleted API and Recover from DeletedException

Posted by "prateekm (via GitHub)" <gi...@apache.org>.
prateekm commented on code in PR #1676:
URL: https://github.com/apache/samza/pull/1676#discussion_r1287527272


##########
samza-api/src/main/java/org/apache/samza/storage/blobstore/BlobStoreManager.java:
##########
@@ -49,11 +49,12 @@ public interface BlobStoreManager {
    * @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
+   * @param getDeletedBlob Flag to indicate if get should try to get a blob marked for deletion but not yet compacted
    * @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);
+  CompletionStage<Void> get(String id, OutputStream outputStream, Metadata metadata, Boolean getDeletedBlob);

Review Comment:
   Minor: boolean



##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreRestoreManager.java:
##########
@@ -119,13 +119,18 @@ public BlobStoreRestoreManager(TaskModel taskModel, ExecutorService restoreExecu
 
   @Override
   public void init(Checkpoint checkpoint) {
+    init(checkpoint, false);
+  }
+
+  public void init(Checkpoint checkpoint, Boolean getDeletedBlob) {

Review Comment:
   Minor: boolean everywhere (vs Boolean).



##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreBackupManager.java:
##########
@@ -139,8 +139,10 @@ public void init(Checkpoint checkpoint) {
 
     // Note: blocks the caller thread.
     // TODO LOW shesharma exclude stores that are no longer configured during init
+    // NOTE: Always get SnapshotIndex with getDeleted enabled. A failure to get a blob from SnapshotIndex would restart

Review Comment:
   Minor: Comment isn't very clear. Doesn't explain _why_ this should be true. Only (if I understand correctly) what happens when this is true and there is still a failure.



##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreRestoreManager.java:
##########
@@ -212,9 +213,17 @@ static CompletableFuture<Void> restoreStores(String jobName, String jobId, TaskN
       File loggedBaseDir, StorageConfig storageConfig, BlobStoreRestoreManagerMetrics metrics,
       StorageManagerUtil storageManagerUtil, BlobStoreUtil blobStoreUtil, DirDiffUtil dirDiffUtil,
       ExecutorService executor) {
+    return restoreStores(jobName, jobId, taskName, storesToRestore, prevStoreSnapshotIndexes, loggedBaseDir, storageConfig,
+        metrics, storageManagerUtil, blobStoreUtil, dirDiffUtil, executor, false);
+  }
+
+  public static CompletableFuture<Void> restoreStores(String jobName, String jobId, TaskName taskName, Set<String> storesToRestore,

Review Comment:
   Minor: Why public? Make package private and @VisibleForTesting?



##########
samza-core/src/main/java/org/apache/samza/storage/TaskStorageCommitManager.java:
##########
@@ -93,14 +93,18 @@ public TaskStorageCommitManager(TaskName taskName, Map<String, TaskBackupManager
     this.metrics = metrics;
   }
 
-  public void init() {
+  public void init(Checkpoint checkpoint) {
     // Assuming that container storage manager has already started and created to stores
     storageEngines = containerStorageManager.getAllStores(taskName);
     if (checkpointManager != null) {
-      Checkpoint checkpoint = checkpointManager.readLastCheckpoint(taskName);
+      // In case of standby containers

Review Comment:
   Minor: Explain inline why this is the case for standby containers.



##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreRestoreManager.java:
##########
@@ -254,8 +263,10 @@ static CompletableFuture<Void> restoreStores(String jobName, String jobId, TaskN
         throw new SamzaException(String.format("Error deleting store directory: %s", storeDir), e);
       }
 
+      // If getDeletedBlobs is enabled - always restore so that we get all the blobs, including the deleted blobs,
+      // immediately restore it locally and backup to create new checkpoint.

Review Comment:
   Minor: "backup to create .." is not happening here, clarify comment.



##########
samza-core/src/main/java/org/apache/samza/storage/TaskStorageCommitManager.java:
##########
@@ -93,14 +93,18 @@ public TaskStorageCommitManager(TaskName taskName, Map<String, TaskBackupManager
     this.metrics = metrics;
   }
 
-  public void init() {
+  public void init(Checkpoint checkpoint) {
     // Assuming that container storage manager has already started and created to stores
     storageEngines = containerStorageManager.getAllStores(taskName);
     if (checkpointManager != null) {
-      Checkpoint checkpoint = checkpointManager.readLastCheckpoint(taskName);
+      // In case of standby containers
+      if (checkpoint == null) {
+        checkpoint = checkpointManager.readLastCheckpoint(taskName);

Review Comment:
   Minor: prefer not reassigning input parameter. Create finalCheckpoint (maybe rename to latestCheckpoint) earlier and assign/reassign to it.



##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/util/BlobStoreUtil.java:
##########
@@ -210,6 +216,56 @@ public CompletableFuture<String> putSnapshotIndex(SnapshotIndex snapshotIndex) {
     }, isCauseNonRetriable(), executor, retryPolicyConfig);
   }
 
+  /**
+   * Cleans up a SnapshotIndex by recursively deleting all blobs associated with files/subdirs inside the SnapshotIndex
+   * and finally deletes SnapshotIndex blob itself. This is done by getting the SnapshotIndex first.
+   * @param snapshotIndexBlobId Blob if of SnapshotIndex
+   * @param requestMetadata Metadata of the request
+   */
+  public CompletionStage<Void> cleanSnapshotIndex(String snapshotIndexBlobId, Metadata requestMetadata) {
+    return cleanSnapshotIndex(snapshotIndexBlobId, requestMetadata, false);
+  }
+
+  /**
+   * Cleans up a SnapshotIndex by recursively deleting all blobs associated with files/subdirs inside the SnapshotIndex
+   * and finally deletes SnapshotIndex blob itself. This is done by getting the SnapshotIndex first.
+   * @param snapshotIndexBlobId Blob if of SnapshotIndex
+   * @param requestMetadata Metadata of the request
+   * @param getDeletedBlob Gets SnapshotIndex with getDeleted flag set
+   */
+  public CompletionStage<Void> cleanSnapshotIndex(String snapshotIndexBlobId, Metadata requestMetadata, Boolean getDeletedBlob) {
+    Metadata getSnapshotRequest = new Metadata(Metadata.SNAPSHOT_INDEX_PAYLOAD_PATH, Optional.empty(), requestMetadata.getJobName(),
+        requestMetadata.getJobId(), requestMetadata.getTaskName(), requestMetadata.getStoreName());
+    SnapshotIndex snapshotIndex = getSnapshotIndex(snapshotIndexBlobId, getSnapshotRequest, getDeletedBlob).join();
+    return cleanSnapshotIndex(snapshotIndexBlobId, snapshotIndex, requestMetadata);
+  }
+
+  /**
+   * Cleans up a SnapshotIndex by recursively deleting all blobs associated with files/subdirs inside the SnapshotIndex
+   * and finally deletes SnapshotIndex blob itself.
+   * @param snapshotIndexBlobId Blob if of SnapshotIndex
+   * @param snapshotIndex SnapshotIndex to delete
+   * @param requestMetadata Metadata of the request
+   */
+  public CompletionStage<Void> cleanSnapshotIndex(String snapshotIndexBlobId, SnapshotIndex snapshotIndex, Metadata requestMetadata) {
+    DirIndex dirIndex = snapshotIndex.getDirIndex();
+    CompletionStage<Void> storeDeletionFuture =
+        cleanUpDir(dirIndex, requestMetadata) // delete files and sub-dirs previously marked for removal
+            .thenComposeAsync(v ->
+                deleteDir(dirIndex, requestMetadata), executor) // deleted files and dirs still present
+            .thenComposeAsync(v -> deleteSnapshotIndexBlob(snapshotIndexBlobId, requestMetadata), executor) // delete the snapshot index blob
+            .exceptionally(ex -> {
+              if (ex instanceof DeletedException) {

Review Comment:
   Doesn't look like it's fixed.



##########
samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala:
##########
@@ -149,31 +149,33 @@ class TaskInstance(
     }
   }
 
-  def initTask {
+  def initTask(lastTaskCheckpoint: Checkpoint) {
     initCaughtUpMapping()
 
+    val isStandByTask = taskModel.getTaskMode == TaskMode.Standby
+
     if (commitManager != null) {
       debug("Starting commit manager for taskName: %s" format taskName)
-
-      commitManager.init()
+      commitManager.init(if (isStandByTask) null else lastTaskCheckpoint)

Review Comment:
   Doesn't seem to be fixed. Missing the latest commit?



##########
samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala:
##########
@@ -981,7 +981,7 @@ class SamzaContainer(
     }
   }
 
-  def startStores {
+  def startStores: util.Map[TaskName, Checkpoint] = {

Review Comment:
   Doesn't seem to be fixed.



##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/util/BlobStoreUtil.java:
##########
@@ -114,6 +114,11 @@ public BlobStoreUtil(BlobStoreManager blobStoreManager, ExecutorService executor
    */
   public Map<String, Pair<String, SnapshotIndex>> getStoreSnapshotIndexes(
       String jobName, String jobId, String taskName, Checkpoint checkpoint, Set<String> storesToBackupOrRestore) {
+    return getStoreSnapshotIndexes(jobName, jobId, taskName, checkpoint, storesToBackupOrRestore, false);
+  }
+
+  public Map<String, Pair<String, SnapshotIndex>> getStoreSnapshotIndexes(String jobName, String jobId, String taskName,

Review Comment:
   See comments in BSRM and BSBM init. When do we need to get snapshot indexes with getDeletedBlobs = false?



##########
samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala:
##########
@@ -149,31 +149,33 @@ class TaskInstance(
     }
   }
 
-  def initTask {
+  def initTask(lastTaskCheckpoint: Checkpoint) {

Review Comment:
   Doesn't seem to be fixed.



##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManagerUtil.java:
##########
@@ -411,4 +438,292 @@ public static Set<String> getSideInputStoreNames(
     }
     return sideInputStores;
   }
+
+  public static List<Future<Void>> initAndRestoreTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, SamzaContainerMetrics samzaContainerMetrics,
+      CheckpointManager checkpointManager, JobContext jobContext, ContainerModel containerModel,
+      Map<TaskName, Checkpoint> taskCheckpoints, Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames,
+      Config config, ExecutorService executor, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      File loggerStoreDir, Map<String, SystemConsumer> storeConsumers) {
+
+    Set<String> forceRestoreTasks = new HashSet<>();
+    // Initialize each TaskStorageManager.
+    taskRestoreManagers.forEach((taskName, restoreManagers) ->
+        restoreManagers.forEach((factoryName, taskRestoreManager) -> {
+          try {
+            taskRestoreManager.init(taskCheckpoints.get(taskName));
+          } catch (SamzaException ex) {
+            if (isUnwrappedExceptionDeletedException(ex) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+              // Get deleted SnapshotIndex blob with GetDeleted and mark the task to be restored with GetDeleted as well.
+              // this ensures that the restore downloads the snapshot, recreates a new snapshot, uploads it to blob store
+              // and creates a new checkpoint.
+              ((BlobStoreRestoreManager) taskRestoreManager).init(taskCheckpoints.get(taskName), true);

Review Comment:
   See comments above in BSRM and BSBM init. We may not need to call init twice (or with getDeleted) at all.



##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/util/BlobStoreUtil.java:
##########
@@ -210,6 +216,56 @@ public CompletableFuture<String> putSnapshotIndex(SnapshotIndex snapshotIndex) {
     }, isCauseNonRetriable(), executor, retryPolicyConfig);
   }
 
+  /**
+   * Cleans up a SnapshotIndex by recursively deleting all blobs associated with files/subdirs inside the SnapshotIndex
+   * and finally deletes SnapshotIndex blob itself. This is done by getting the SnapshotIndex first.
+   * @param snapshotIndexBlobId Blob if of SnapshotIndex
+   * @param requestMetadata Metadata of the request
+   */
+  public CompletionStage<Void> cleanSnapshotIndex(String snapshotIndexBlobId, Metadata requestMetadata) {
+    return cleanSnapshotIndex(snapshotIndexBlobId, requestMetadata, false);
+  }
+
+  /**
+   * Cleans up a SnapshotIndex by recursively deleting all blobs associated with files/subdirs inside the SnapshotIndex
+   * and finally deletes SnapshotIndex blob itself. This is done by getting the SnapshotIndex first.
+   * @param snapshotIndexBlobId Blob if of SnapshotIndex
+   * @param requestMetadata Metadata of the request
+   * @param getDeletedBlob Gets SnapshotIndex with getDeleted flag set
+   */
+  public CompletionStage<Void> cleanSnapshotIndex(String snapshotIndexBlobId, Metadata requestMetadata, Boolean getDeletedBlob) {
+    Metadata getSnapshotRequest = new Metadata(Metadata.SNAPSHOT_INDEX_PAYLOAD_PATH, Optional.empty(), requestMetadata.getJobName(),
+        requestMetadata.getJobId(), requestMetadata.getTaskName(), requestMetadata.getStoreName());
+    SnapshotIndex snapshotIndex = getSnapshotIndex(snapshotIndexBlobId, getSnapshotRequest, getDeletedBlob).join();

Review Comment:
   Why join instead of chain the next call?



##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/util/BlobStoreUtil.java:
##########
@@ -232,6 +288,15 @@ public CompletionStage<Void> deleteSnapshotIndexBlob(String snapshotIndexBlobId,
    * @return A future that completes when all the async downloads completes
    */
   public CompletableFuture<Void> restoreDir(File baseDir, DirIndex dirIndex, Metadata metadata) {
+    return restoreDir(baseDir, dirIndex, metadata, false);
+  }
+
+  /**
+   * Non-blocking restore of a {@link SnapshotIndex} to local store by downloading all the files and sub-dirs associated
+   * with this remote snapshot. getDeletedFiles flag sets whether to attempt a get for deletedFiles or not.
+   * @return A future that completes when all the async downloads completes
+   */
+  public CompletableFuture<Void> restoreDir(File baseDir, DirIndex dirIndex, Metadata metadata, Boolean getDeletedFiles) {

Review Comment:
   Don't need 2 variants for the util method.



##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreRestoreManager.java:
##########
@@ -119,13 +119,18 @@ public BlobStoreRestoreManager(TaskModel taskModel, ExecutorService restoreExecu
 
   @Override
   public void init(Checkpoint checkpoint) {
+    init(checkpoint, false);
+  }
+
+  public void init(Checkpoint checkpoint, Boolean getDeletedBlob) {
     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, storesToRestore);
+    prevStoreSnapshotIndexes =
+        blobStoreUtil.getStoreSnapshotIndexes(jobName, jobId, taskName, checkpoint, storesToRestore, getDeletedBlob);

Review Comment:
   Why not always get snapshot index with getDeletedBlob = true in init (and remove this variant)?
   
   IIUC (correct me if wrong):
   1. We only getStoreSnapshotIndex twice during container startup, once in BSRestoreManager, once in BSBackupManager.
   2. Either both requests with getDeleted will succeed, or not.
           a. If both succeed, then regardless of whether the snapshot index blob is deleted or not, restore and future backups will continue uninterrupted. If the restore for an actual file in the snapshot fails, then BSRM will retry the entire restore() with getDeletedBlob = true with the same snapshot index.
           b. If either fails, then the snapshot index blob is gone for good and there is nothing we can do.
           
    So why do we need the let the first BSRM init request fail for deleted snapshot index blob?
           



##########
samza-core/src/main/java/org/apache/samza/util/FutureUtil.java:
##########
@@ -139,6 +139,17 @@ public static <K, V> CompletableFuture<Map<K, V>> toFutureOfMap(
     });
   }
 
+  /**
+   * Helper method to convert: {@code Map<K, CompletionStage<Void>>}
+   * to:                       {@code CompletableFuture<Void>}
+   *
+   * Returns a future that completes when all value futures complete.
+   * Returned future completes exceptionally if any of the value futures complete exceptionally.
+   */
+  public static CompletableFuture<Void> mapToFuture(Map<String, CompletionStage<Void>> map) {

Review Comment:
   Can we not just ignore the return value in the previous method?



##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreRestoreManager.java:
##########
@@ -254,8 +263,10 @@ static CompletableFuture<Void> restoreStores(String jobName, String jobId, TaskN
         throw new SamzaException(String.format("Error deleting store directory: %s", storeDir), e);
       }
 
+      // If getDeletedBlobs is enabled - always restore so that we get all the blobs, including the deleted blobs,
+      // immediately restore it locally and backup to create new checkpoint.
       boolean shouldRestore = shouldRestore(taskName.getTaskName(), storeName, dirIndex,
-          storeCheckpointDir, storageConfig, dirDiffUtil);
+          storeCheckpointDir, storageConfig, dirDiffUtil) || getDeletedBlobs;

Review Comment:
   If shouldRestore returned false during first restore, there would have been no restore, no failures, and we would not be restoring with getDeleted = true. 
   
   E.g. if the last checkpoint directory was already present and was identical to remote snapshot, it would be copied over to the store directory and restore would be skipped. 
   
   It seems like this would be an issue, since the files could actually be deleted in remote store, but restore manager wouldn't realize it, and backup manager would only upload the delta based on the latest snapshot index. So the files could eventually get compacted.
   
   So do we need to remove the shouldRestore check and always force a restore upon startup to detect correctly if we need to reupload?
   
   Can sync up offline if I'm misunderstanding something.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] shekhars-li commented on a diff in pull request #1676: [SAMZA-2787] GetDeleted API and Recover from DeletedException

Posted by "shekhars-li (via GitHub)" <gi...@apache.org>.
shekhars-li commented on code in PR #1676:
URL: https://github.com/apache/samza/pull/1676#discussion_r1287911253


##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManagerUtil.java:
##########
@@ -411,4 +438,292 @@ public static Set<String> getSideInputStoreNames(
     }
     return sideInputStores;
   }
+
+  public static List<Future<Void>> initAndRestoreTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, SamzaContainerMetrics samzaContainerMetrics,
+      CheckpointManager checkpointManager, JobContext jobContext, ContainerModel containerModel,
+      Map<TaskName, Checkpoint> taskCheckpoints, Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames,
+      Config config, ExecutorService executor, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      File loggerStoreDir, Map<String, SystemConsumer> storeConsumers) {
+
+    Set<String> forceRestoreTasks = new HashSet<>();
+    // Initialize each TaskStorageManager.
+    taskRestoreManagers.forEach((taskName, restoreManagers) ->
+        restoreManagers.forEach((factoryName, taskRestoreManager) -> {
+          try {
+            taskRestoreManager.init(taskCheckpoints.get(taskName));
+          } catch (SamzaException ex) {
+            if (isUnwrappedExceptionDeletedException(ex) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+              // Get deleted SnapshotIndex blob with GetDeleted and mark the task to be restored with GetDeleted as well.
+              // this ensures that the restore downloads the snapshot, recreates a new snapshot, uploads it to blob store
+              // and creates a new checkpoint.
+              ((BlobStoreRestoreManager) taskRestoreManager).init(taskCheckpoints.get(taskName), true);
+              forceRestoreTasks.add(taskName.getTaskName());
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("init failed for task: %s with GetDeleted set to true", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex);
+            }
+          }
+        })
+    );
+
+    // Start each store consumer once.
+    // Note: These consumers are per system and only changelog system store consumers will be started.
+    // Some TaskRestoreManagers may not require the consumer to be started, but due to the agnostic nature of
+    // ContainerStorageManager we always start the changelog consumer here in case it is required
+    storeConsumers.values().stream().distinct().forEach(SystemConsumer::start);
+
+    return restoreAllTaskInstances(taskRestoreManagers, taskCheckpoints, taskBackendFactoryToStoreNames, jobContext,
+        containerModel, samzaContainerMetrics, checkpointManager, config, taskInstanceMetrics, executor, loggerStoreDir,
+        forceRestoreTasks);
+  }
+
+  /**
+   * Restores all TaskInstances and returns a future for each TaskInstance restore. If this restore fails with DeletedException
+   * it will retry the restore with getDeleted flag, get all the blobs, recreate a new checkpoint in blob store and
+   * write that checkpoint to checkpoint topic.
+   * @param taskRestoreManagers map of TaskName to factory name to TaskRestoreManager map.
+   */
+  public static List<Future<Void>> restoreAllTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, Map<TaskName, Checkpoint> taskCheckpoints,
+      Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames, JobContext jobContext,
+      ContainerModel containerModel, SamzaContainerMetrics samzaContainerMetrics, CheckpointManager checkpointManager,
+      Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics, ExecutorService executor,
+      File loggedStoreDir, Set<String> forceRestoreTask) {
+
+    List<Future<Void>> taskRestoreFutures = new ArrayList<>();
+
+    // Submit restore callable for each taskInstance
+    taskRestoreManagers.forEach((taskInstanceName, restoreManagersMap) -> {
+      // Submit for each restore factory
+      restoreManagersMap.forEach((factoryName, taskRestoreManager) -> {
+        long startTime = System.currentTimeMillis();
+        String taskName = taskInstanceName.getTaskName();
+        LOG.info("Starting restore for state for task: {}", taskName);
+
+        CompletableFuture<Void> restoreFuture;
+        if (forceRestoreTask.contains(taskName) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+          restoreFuture = ((BlobStoreRestoreManager) taskRestoreManager).restore(true);
+        } else {
+          restoreFuture = taskRestoreManager.restore();
+        }
+
+        CompletableFuture<Void> taskRestoreFuture = restoreFuture.handle((res, ex) -> {
+          updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+
+          if (ex != null) {
+            if (isUnwrappedExceptionDeletedException(ex)) {
+              LOG.warn(
+                  "Restore state for task: {} received DeletedException. Reattempting with getDeletedBlobs enabled",
+                  taskInstanceName.getTaskName());
+
+              // Try to restore with getDeleted flag
+              CompletableFuture<Void> future =
+                  restoreTaskStoresAndCreateCheckpointWithGetDeleted(taskInstanceName, taskCheckpoints, checkpointManager,
+                      taskRestoreManager, config, taskInstanceMetrics, executor,
+                      taskBackendFactoryToStoreNames.get(taskInstanceName).get(factoryName), loggedStoreDir, jobContext,
+                      containerModel);
+              try {
+                if (future != null) {
+                  future.join(); // Block and restore deleted state before continuing
+                }
+              } catch (Exception e) {
+                String msg = String.format("Unable to recover from DeletedException for task %s.", taskName);
+                throw new SamzaException(msg, e);
+              } finally {
+                updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+              }
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("Error restoring state for task: %s", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex); // wrap in unchecked exception to throw from lambda
+            }
+          }
+
+          // Stop all persistent stores after restoring. Certain persistent stores opened in BulkLoad mode are compacted
+          // on stop, so paralleling stop() also parallelizes their compaction (a time-intensive operation).
+          try {
+            taskRestoreManager.close();
+          } catch (Exception e) {
+            LOG.error("Error closing restore manager for task: {} after {} restore", taskName,
+                ex != null ? "unsuccessful" : "successful", e);
+            // ignore exception from close. container may still be able to continue processing/backups
+            // if restore manager close fails.
+          }
+          return null;
+        });
+        taskRestoreFutures.add(taskRestoreFuture);
+      });
+    });
+
+    return taskRestoreFutures;
+  }
+
+  public static CompletableFuture<Void> restoreTaskStoresAndCreateCheckpointWithGetDeleted(TaskName taskName,
+      Map<TaskName, Checkpoint> taskCheckpoints, CheckpointManager checkpointManager,
+      TaskRestoreManager taskRestoreManager, Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      ExecutorService executor, Set<String> storesToRestore, File loggedStoreBaseDirectory, JobContext jobContext,
+      ContainerModel containerModel) {
+
+    // if taskInstanceMetrics are specified use those for store metrics,
+    // otherwise (in case of StorageRecovery) use a blank MetricsRegistryMap
+    MetricsRegistry metricsRegistry =
+        taskInstanceMetrics.get(taskName) != null ? taskInstanceMetrics.get(taskName).registry()
+            : new MetricsRegistryMap();
+
+    BlobStoreManager blobStoreManager = getBlobStoreManager(config, executor);
+    JobConfig jobConfig = new JobConfig(config);
+    BlobStoreUtil blobStoreUtil =
+        new BlobStoreUtil(blobStoreManager, executor, new BlobStoreConfig(config), null,
+            new BlobStoreRestoreManagerMetrics(metricsRegistry));
+
+    BlobStoreRestoreManager blobStoreRestoreManager = (BlobStoreRestoreManager) taskRestoreManager;
+
+    CheckpointId checkpointId = CheckpointId.create();
+    Map<String, String> oldSCMs = ((CheckpointV2) taskCheckpoints.get(taskName))
+        .getStateCheckpointMarkers().get(BlobStoreStateBackendFactory.class.getName());
+
+    // Returns a single future that guarantees all the following are completed, in this order:
+    // 1. Restore state locally by getting deleted blobs from the blob store.
+    // 2. Create a new Checkpoint from restored state and back it up on the blob store.
+    // 3. Clean up old/deleted Snapshot
+    // 4. Remove TTL from the new Snapshot on the blob store
+    // 5. Write the new checkpoint to checkpoint topic and update taskCheckpoints map

Review Comment:
   Updated that to not mutate anymore.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] prateekm commented on a diff in pull request #1676: [SAMZA-2787] GetDeleted API and Recover from DeletedException

Posted by "prateekm (via GitHub)" <gi...@apache.org>.
prateekm commented on code in PR #1676:
URL: https://github.com/apache/samza/pull/1676#discussion_r1273950510


##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/util/BlobStoreUtil.java:
##########
@@ -114,6 +114,11 @@ public BlobStoreUtil(BlobStoreManager blobStoreManager, ExecutorService executor
    */
   public Map<String, Pair<String, SnapshotIndex>> getStoreSnapshotIndexes(
       String jobName, String jobId, String taskName, Checkpoint checkpoint, Set<String> storesToBackupOrRestore) {
+    return getStoreSnapshotIndexes(jobName, jobId, taskName, checkpoint, storesToBackupOrRestore, false);
+  }
+
+  public Map<String, Pair<String, SnapshotIndex>> getStoreSnapshotIndexes(String jobName, String jobId, String taskName,

Review Comment:
   Same as other comments re: variants.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] prateekm commented on a diff in pull request #1676: [SAMZA-2787] GetDeleted API and Recover from DeletedException

Posted by "prateekm (via GitHub)" <gi...@apache.org>.
prateekm commented on code in PR #1676:
URL: https://github.com/apache/samza/pull/1676#discussion_r1288932588


##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreRestoreManager.java:
##########
@@ -254,8 +263,10 @@ static CompletableFuture<Void> restoreStores(String jobName, String jobId, TaskN
         throw new SamzaException(String.format("Error deleting store directory: %s", storeDir), e);
       }
 
+      // If getDeletedBlobs is enabled - always restore so that we get all the blobs, including the deleted blobs,
+      // immediately restore it locally and backup to create new checkpoint.
       boolean shouldRestore = shouldRestore(taskName.getTaskName(), storeName, dirIndex,
-          storeCheckpointDir, storageConfig, dirDiffUtil);
+          storeCheckpointDir, storageConfig, dirDiffUtil) || getDeletedBlobs;

Review Comment:
   Yes, makes sense. To close the loop:
   Snapshot index blob is deleted after all no-longer-needed file blobs in it are deleted.
   We are using the failure to get snapshot index in init with getDeletedBlob = false as the signal that some of its contained blobs are missing and the entire store needs to be restored. This is also why the higher precedence for getDeletedBlobs is required.
   For the rare race condition where file blobs were deleted but snapshot index wasn't yet, restore manager init will work, restore will work, but a subsequent commit will fail and restart the container. By that time, the SnapshotIndex should be deleted by the orphaned container.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] shekhars-li commented on a diff in pull request #1676: [SAMZA-2787] GetDeleted API and Recover from DeletedException

Posted by "shekhars-li (via GitHub)" <gi...@apache.org>.
shekhars-li commented on code in PR #1676:
URL: https://github.com/apache/samza/pull/1676#discussion_r1286390522


##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java:
##########
@@ -301,64 +300,14 @@ private void restoreStores() throws InterruptedException {
               samzaContainerMetrics, taskInstanceMetrics, taskInstanceCollectors, serdes,
               loggedStoreBaseDirectory, nonLoggedStoreBaseDirectory, config, clock);
       taskRestoreManagers.put(taskName, taskStoreRestoreManagers);
+      taskBackendFactoryToStoreNames.put(taskName, backendFactoryToStoreNames);
     });
 
-    // Initialize each TaskStorageManager
-    taskRestoreManagers.forEach((taskName, restoreManagers) ->
-        restoreManagers.forEach((factoryName, taskRestoreManager) ->
-            taskRestoreManager.init(taskCheckpoints.get(taskName))
-        )
-    );
-
-    // Start each store consumer once.
-    // Note: These consumers are per system and only changelog system store consumers will be started.
-    // Some TaskRestoreManagers may not require the consumer to to be started, but due to the agnostic nature of
-    // ContainerStorageManager we always start the changelog consumer here in case it is required
-    this.storeConsumers.values().stream().distinct().forEach(SystemConsumer::start);
-
-    List<Future<Void>> taskRestoreFutures = new ArrayList<>();
-
-    // Submit restore callable for each taskInstance
-    taskRestoreManagers.forEach((taskInstance, restoreManagersMap) -> {
-      // Submit for each restore factory
-      restoreManagersMap.forEach((factoryName, taskRestoreManager) -> {
-        long startTime = System.currentTimeMillis();
-        String taskName = taskInstance.getTaskName();
-        LOG.info("Starting restore for state for task: {}", taskName);
-        CompletableFuture<Void> restoreFuture = taskRestoreManager.restore().handle((res, ex) -> {
-          // Stop all persistent stores after restoring. Certain persistent stores opened in BulkLoad mode are compacted
-          // on stop, so paralleling stop() also parallelizes their compaction (a time-intensive operation).
-          try {
-            taskRestoreManager.close();
-          } catch (Exception e) {
-            LOG.error("Error closing restore manager for task: {} after {} restore",
-                taskName, ex != null ? "unsuccessful" : "successful", e);
-            // ignore exception from close. container may still be be able to continue processing/backups
-            // if restore manager close fails.
-          }
-
-          long timeToRestore = System.currentTimeMillis() - startTime;
-          if (samzaContainerMetrics != null) {
-            Gauge taskGauge = samzaContainerMetrics.taskStoreRestorationMetrics().getOrDefault(taskInstance, null);
-
-            if (taskGauge != null) {
-              taskGauge.set(timeToRestore);
-            }
-          }
-
-          if (ex != null) {
-            // log and rethrow exception to communicate restore failure
-            String msg = String.format("Error restoring state for task: %s", taskName);
-            LOG.error(msg, ex);
-            throw new SamzaException(msg, ex); // wrap in unchecked exception to throw from lambda
-          } else {
-            return null;
-          }
-        });
-
-        taskRestoreFutures.add(restoreFuture);
-      });
-    });
+    // Init all taskRestores and if successful, create a future for restores for each task
+    List<Future<Void>> taskRestoreFutures =

Review Comment:
   Updated to now return a map of task name and checkpoints rather than mutating the input 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] shekhars-li commented on a diff in pull request #1676: [SAMZA-2787] GetDeleted API and Recover from DeletedException

Posted by "shekhars-li (via GitHub)" <gi...@apache.org>.
shekhars-li commented on code in PR #1676:
URL: https://github.com/apache/samza/pull/1676#discussion_r1286271287


##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/util/BlobStoreUtil.java:
##########
@@ -210,6 +216,56 @@ public CompletableFuture<String> putSnapshotIndex(SnapshotIndex snapshotIndex) {
     }, isCauseNonRetriable(), executor, retryPolicyConfig);
   }
 
+  /**
+   * Cleans up a SnapshotIndex by recursively deleting all blobs associated with files/subdirs inside the SnapshotIndex
+   * and finally deletes SnapshotIndex blob itself. This is done by getting the SnapshotIndex first.
+   * @param snapshotIndexBlobId Blob if of SnapshotIndex
+   * @param requestMetadata Metadata of the request
+   */
+  public CompletionStage<Void> cleanSnapshotIndex(String snapshotIndexBlobId, Metadata requestMetadata) {
+    return cleanSnapshotIndex(snapshotIndexBlobId, requestMetadata, false);
+  }
+
+  /**
+   * Cleans up a SnapshotIndex by recursively deleting all blobs associated with files/subdirs inside the SnapshotIndex
+   * and finally deletes SnapshotIndex blob itself. This is done by getting the SnapshotIndex first.
+   * @param snapshotIndexBlobId Blob if of SnapshotIndex
+   * @param requestMetadata Metadata of the request
+   * @param getDeletedBlob Gets SnapshotIndex with getDeleted flag set
+   */
+  public CompletionStage<Void> cleanSnapshotIndex(String snapshotIndexBlobId, Metadata requestMetadata, Boolean getDeletedBlob) {
+    Metadata getSnapshotRequest = new Metadata(Metadata.SNAPSHOT_INDEX_PAYLOAD_PATH, Optional.empty(), requestMetadata.getJobName(),
+        requestMetadata.getJobId(), requestMetadata.getTaskName(), requestMetadata.getStoreName());
+    SnapshotIndex snapshotIndex = getSnapshotIndex(snapshotIndexBlobId, getSnapshotRequest, getDeletedBlob).join();
+    return cleanSnapshotIndex(snapshotIndexBlobId, snapshotIndex, requestMetadata);
+  }
+
+  /**
+   * Cleans up a SnapshotIndex by recursively deleting all blobs associated with files/subdirs inside the SnapshotIndex
+   * and finally deletes SnapshotIndex blob itself.
+   * @param snapshotIndexBlobId Blob if of SnapshotIndex
+   * @param snapshotIndex SnapshotIndex to delete
+   * @param requestMetadata Metadata of the request
+   */
+  public CompletionStage<Void> cleanSnapshotIndex(String snapshotIndexBlobId, SnapshotIndex snapshotIndex, Metadata requestMetadata) {
+    DirIndex dirIndex = snapshotIndex.getDirIndex();
+    CompletionStage<Void> storeDeletionFuture =
+        cleanUpDir(dirIndex, requestMetadata) // delete files and sub-dirs previously marked for removal
+            .thenComposeAsync(v ->
+                deleteDir(dirIndex, requestMetadata), executor) // deleted files and dirs still present
+            .thenComposeAsync(v -> deleteSnapshotIndexBlob(snapshotIndexBlobId, requestMetadata), executor) // delete the snapshot index blob
+            .exceptionally(ex -> {
+              if (ex instanceof DeletedException) {
+                LOG.warn("DeletedException received on trying to clean up SnapshotIndex {}. Ignoring the error.",

Review Comment:
   Added request metadata, that should have all the necessary info



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] shekhars-li commented on a diff in pull request #1676: [SAMZA-2787] GetDeleted API and Recover from DeletedException

Posted by "shekhars-li (via GitHub)" <gi...@apache.org>.
shekhars-li commented on code in PR #1676:
URL: https://github.com/apache/samza/pull/1676#discussion_r1286263514


##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreRestoreManager.java:
##########
@@ -212,9 +213,17 @@ static CompletableFuture<Void> restoreStores(String jobName, String jobId, TaskN
       File loggedBaseDir, StorageConfig storageConfig, BlobStoreRestoreManagerMetrics metrics,
       StorageManagerUtil storageManagerUtil, BlobStoreUtil blobStoreUtil, DirDiffUtil dirDiffUtil,
       ExecutorService executor) {
+    return restoreStores(jobName, jobId, taskName, storesToRestore, prevStoreSnapshotIndexes, loggedBaseDir, storageConfig,
+        metrics, storageManagerUtil, blobStoreUtil, dirDiffUtil, executor, false);
+  }
+
+  public static CompletableFuture<Void> restoreStores(String jobName, String jobId, TaskName taskName, Set<String> storesToRestore,

Review Comment:
   Removed one version -> we don't need 2 of these. Thanks



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] shekhars-li commented on a diff in pull request #1676: [SAMZA-2787] GetDeleted API and Recover from DeletedException

Posted by "shekhars-li (via GitHub)" <gi...@apache.org>.
shekhars-li commented on code in PR #1676:
URL: https://github.com/apache/samza/pull/1676#discussion_r1287909576


##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/util/BlobStoreUtil.java:
##########
@@ -232,6 +288,15 @@ public CompletionStage<Void> deleteSnapshotIndexBlob(String snapshotIndexBlobId,
    * @return A future that completes when all the async downloads completes
    */
   public CompletableFuture<Void> restoreDir(File baseDir, DirIndex dirIndex, Metadata metadata) {
+    return restoreDir(baseDir, dirIndex, metadata, false);
+  }
+
+  /**
+   * Non-blocking restore of a {@link SnapshotIndex} to local store by downloading all the files and sub-dirs associated
+   * with this remote snapshot. getDeletedFiles flag sets whether to attempt a get for deletedFiles or not.
+   * @return A future that completes when all the async downloads completes
+   */
+  public CompletableFuture<Void> restoreDir(File baseDir, DirIndex dirIndex, Metadata metadata, Boolean getDeletedFiles) {

Review Comment:
   Updated/removed. Thanks



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] prateekm commented on a diff in pull request #1676: [SAMZA-2787] GetDeleted API and Recover from DeletedException

Posted by "prateekm (via GitHub)" <gi...@apache.org>.
prateekm commented on code in PR #1676:
URL: https://github.com/apache/samza/pull/1676#discussion_r1287605254


##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManagerUtil.java:
##########
@@ -411,4 +438,292 @@ public static Set<String> getSideInputStoreNames(
     }
     return sideInputStores;
   }
+
+  public static List<Future<Void>> initAndRestoreTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, SamzaContainerMetrics samzaContainerMetrics,
+      CheckpointManager checkpointManager, JobContext jobContext, ContainerModel containerModel,
+      Map<TaskName, Checkpoint> taskCheckpoints, Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames,
+      Config config, ExecutorService executor, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      File loggerStoreDir, Map<String, SystemConsumer> storeConsumers) {
+
+    Set<String> forceRestoreTasks = new HashSet<>();
+    // Initialize each TaskStorageManager.
+    taskRestoreManagers.forEach((taskName, restoreManagers) ->
+        restoreManagers.forEach((factoryName, taskRestoreManager) -> {
+          try {
+            taskRestoreManager.init(taskCheckpoints.get(taskName));
+          } catch (SamzaException ex) {
+            if (isUnwrappedExceptionDeletedException(ex) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+              // Get deleted SnapshotIndex blob with GetDeleted and mark the task to be restored with GetDeleted as well.
+              // this ensures that the restore downloads the snapshot, recreates a new snapshot, uploads it to blob store
+              // and creates a new checkpoint.
+              ((BlobStoreRestoreManager) taskRestoreManager).init(taskCheckpoints.get(taskName), true);

Review Comment:
   See comments above in BSRM and BSBM init. We may not need to call init twice (or with getDeleted) at all.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] shekhars-li commented on a diff in pull request #1676: [SAMZA-2787] GetDeleted API and Recover from DeletedException

Posted by "shekhars-li (via GitHub)" <gi...@apache.org>.
shekhars-li commented on code in PR #1676:
URL: https://github.com/apache/samza/pull/1676#discussion_r1286390735


##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java:
##########
@@ -400,6 +349,7 @@ private void restoreStores() throws InterruptedException {
     });
 
     LOG.info("Store Restore complete");
+    return taskCheckpoints;

Review Comment:
   Updated to now return a new map of taskname and checkpoint instead of mutating taskCheckpoints



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] shekhars-li commented on a diff in pull request #1676: [SAMZA-2787] GetDeleted API and Recover from DeletedException

Posted by "shekhars-li (via GitHub)" <gi...@apache.org>.
shekhars-li commented on code in PR #1676:
URL: https://github.com/apache/samza/pull/1676#discussion_r1289372784


##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManagerUtil.java:
##########
@@ -411,4 +438,292 @@ public static Set<String> getSideInputStoreNames(
     }
     return sideInputStores;
   }
+
+  /**
+   * Inits and Restores all the task stores.
+   * Note: In case of {@link BlobStoreRestoreManager}, this method retries init and restore with getDeleted flag if it
+   * receives a {@link DeletedException}. This will create a new checkpoint for the corresponding task.
+   */
+  public static CompletableFuture<Map<TaskName, Checkpoint>> initAndRestoreTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, SamzaContainerMetrics samzaContainerMetrics,
+      CheckpointManager checkpointManager, JobContext jobContext, ContainerModel containerModel,
+      Map<TaskName, Checkpoint> taskCheckpoints, Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames,
+      Config config, ExecutorService executor, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      File loggerStoreDir, Map<String, SystemConsumer> storeConsumers) {
+
+    Set<String> forceRestoreTasks = new HashSet<>();
+    // Initialize each TaskStorageManager.
+    taskRestoreManagers.forEach((taskName, restoreManagers) ->
+        restoreManagers.forEach((factoryName, taskRestoreManager) -> {
+          try {
+            taskRestoreManager.init(taskCheckpoints.get(taskName));
+          } catch (SamzaException ex) {
+            if (isUnwrappedExceptionDeletedException(ex) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+              // Get deleted SnapshotIndex blob with GetDeleted and mark the task to be restored with GetDeleted as well.
+              // this ensures that the restore downloads the snapshot, recreates a new snapshot, uploads it to blob store
+              // and creates a new checkpoint.
+              ((BlobStoreRestoreManager) taskRestoreManager).init(taskCheckpoints.get(taskName), true);
+              forceRestoreTasks.add(taskName.getTaskName());
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("init failed for BlobStoreRestoreManager, task: %s with GetDeleted set to true", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex);
+            }
+          }
+        })
+    );
+
+    // Start each store consumer once.
+    // Note: These consumers are per system and only changelog system store consumers will be started.
+    // Some TaskRestoreManagers may not require the consumer to be started, but due to the agnostic nature of
+    // ContainerStorageManager we always start the changelog consumer here in case it is required
+    storeConsumers.values().stream().distinct().forEach(SystemConsumer::start);
+
+    return restoreAllTaskInstances(taskRestoreManagers, taskCheckpoints, taskBackendFactoryToStoreNames, jobContext,
+        containerModel, samzaContainerMetrics, checkpointManager, config, taskInstanceMetrics, executor, loggerStoreDir,
+        forceRestoreTasks);
+  }
+
+  /**
+   * Restores all TaskInstances and returns a future for each TaskInstance restore. Note: In case of
+   * {@link BlobStoreRestoreManager}, this method restore with getDeleted flag if it receives a
+   * {@link DeletedException}. This will create a new Checkpoint.
+   */
+  private static CompletableFuture<Map<TaskName, Checkpoint>> restoreAllTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, Map<TaskName, Checkpoint> taskCheckpoints,
+      Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames, JobContext jobContext,
+      ContainerModel containerModel, SamzaContainerMetrics samzaContainerMetrics, CheckpointManager checkpointManager,
+      Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics, ExecutorService executor,
+      File loggedStoreDir, Set<String> forceRestoreTask) {
+
+    Map<TaskName, CompletableFuture<Checkpoint>> newTaskCheckpoints = new ConcurrentHashMap<>();
+    List<Future<Void>> taskRestoreFutures = new ArrayList<>();
+
+    // Submit restore callable for each taskInstance
+    taskRestoreManagers.forEach((taskInstanceName, restoreManagersMap) -> {
+      // Submit for each restore factory
+      restoreManagersMap.forEach((factoryName, taskRestoreManager) -> {
+        long startTime = System.currentTimeMillis();
+        String taskName = taskInstanceName.getTaskName();
+        LOG.info("Starting restore for state for task: {}", taskName);
+
+        CompletableFuture<Void> restoreFuture;
+        if (forceRestoreTask.contains(taskName) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+          // If init was retried with getDeleted, force restore with getDeleted as well, since init only inits the
+          // restoreManager with deleted SnapshotIndex but does not retry to recover the deleted blobs and delegates it
+          // to restore().
+          // Create an empty future that fails immediately with DeletedException to force retry in restore.
+          restoreFuture = new CompletableFuture<>();
+          restoreFuture.completeExceptionally(new SamzaException(new DeletedException()));
+        } else {
+          restoreFuture = taskRestoreManager.restore();
+        }
+
+        CompletableFuture<Void> taskRestoreFuture = restoreFuture.handle((res, ex) -> {
+          updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+
+          if (ex != null) {
+            if (isUnwrappedExceptionDeletedException(ex)) {
+              LOG.warn(
+                  "Received DeletedException during restore for task {}. Reattempting to get deleted blobs",
+                  taskInstanceName.getTaskName());
+
+              // Try to restore with getDeleted flag
+              CompletableFuture<Checkpoint> future =
+                  restoreDeletedSnapshot(taskInstanceName, taskCheckpoints,
+                      checkpointManager, taskRestoreManager, config, taskInstanceMetrics, executor,
+                      taskBackendFactoryToStoreNames.get(taskInstanceName).get(factoryName), loggedStoreDir,
+                      jobContext, containerModel);
+              try {
+                if (future != null) {
+                  newTaskCheckpoints.put(taskInstanceName, future);
+                }
+              } catch (Exception e) {

Review Comment:
   Actually, ended up adding try catch after some testing. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] prateekm commented on a diff in pull request #1676: [SAMZA-2787] GetDeleted API and Recover from DeletedException

Posted by "prateekm (via GitHub)" <gi...@apache.org>.
prateekm commented on code in PR #1676:
URL: https://github.com/apache/samza/pull/1676#discussion_r1288871486


##########
samza-core/src/main/java/org/apache/samza/storage/TaskStorageCommitManager.java:
##########
@@ -93,18 +93,16 @@ public TaskStorageCommitManager(TaskName taskName, Map<String, TaskBackupManager
     this.metrics = metrics;
   }
 
-  public void init() {
+  public void init(Checkpoint checkpoint) {
     // Assuming that container storage manager has already started and created to stores
     storageEngines = containerStorageManager.getAllStores(taskName);
-    if (checkpointManager != null) {
-      Checkpoint checkpoint = checkpointManager.readLastCheckpoint(taskName);
-      LOG.debug("Last checkpoint on start for task: {} is: {}", taskName, checkpoint);
-      stateBackendToBackupManager.values()
-          .forEach(storageBackupManager -> storageBackupManager.init(checkpoint));
-    } else {
-      stateBackendToBackupManager.values()
-          .forEach(storageBackupManager -> storageBackupManager.init(null));
+    final Checkpoint[] finalCheckpoint = new Checkpoint[]{checkpoint};

Review Comment:
   Minor: Prefer just naming this latestCheckpoint instead of doing this.



##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/util/BlobStoreUtil.java:
##########
@@ -264,7 +303,7 @@ public CompletableFuture<Void> restoreDir(File baseDir, DirIndex dirIndex, Metad
     List<DirIndex> subDirs = dirIndex.getSubDirsPresent();
     for (DirIndex subDir : subDirs) {
       File subDirFile = Paths.get(baseDir.getAbsolutePath(), subDir.getDirName()).toFile();
-      downloadFutures.add(restoreDir(subDirFile, subDir, metadata));
+      downloadFutures.add(restoreDir(subDirFile, subDir, metadata, false));

Review Comment:
   Blocker: This should use the boolean flag too.



##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManagerUtil.java:
##########
@@ -411,4 +438,292 @@ public static Set<String> getSideInputStoreNames(
     }
     return sideInputStores;
   }
+
+  /**
+   * Inits and Restores all the task stores.
+   * Note: In case of {@link BlobStoreRestoreManager}, this method retries init and restore with getDeleted flag if it
+   * receives a {@link DeletedException}. This will create a new checkpoint for the corresponding task.
+   */
+  public static CompletableFuture<Map<TaskName, Checkpoint>> initAndRestoreTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, SamzaContainerMetrics samzaContainerMetrics,
+      CheckpointManager checkpointManager, JobContext jobContext, ContainerModel containerModel,
+      Map<TaskName, Checkpoint> taskCheckpoints, Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames,
+      Config config, ExecutorService executor, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      File loggerStoreDir, Map<String, SystemConsumer> storeConsumers) {
+
+    Set<String> forceRestoreTasks = new HashSet<>();
+    // Initialize each TaskStorageManager.
+    taskRestoreManagers.forEach((taskName, restoreManagers) ->
+        restoreManagers.forEach((factoryName, taskRestoreManager) -> {
+          try {
+            taskRestoreManager.init(taskCheckpoints.get(taskName));
+          } catch (SamzaException ex) {
+            if (isUnwrappedExceptionDeletedException(ex) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+              // Get deleted SnapshotIndex blob with GetDeleted and mark the task to be restored with GetDeleted as well.
+              // this ensures that the restore downloads the snapshot, recreates a new snapshot, uploads it to blob store
+              // and creates a new checkpoint.
+              ((BlobStoreRestoreManager) taskRestoreManager).init(taskCheckpoints.get(taskName), true);
+              forceRestoreTasks.add(taskName.getTaskName());
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("init failed for BlobStoreRestoreManager, task: %s with GetDeleted set to true", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex);
+            }
+          }
+        })
+    );
+
+    // Start each store consumer once.
+    // Note: These consumers are per system and only changelog system store consumers will be started.
+    // Some TaskRestoreManagers may not require the consumer to be started, but due to the agnostic nature of
+    // ContainerStorageManager we always start the changelog consumer here in case it is required
+    storeConsumers.values().stream().distinct().forEach(SystemConsumer::start);
+
+    return restoreAllTaskInstances(taskRestoreManagers, taskCheckpoints, taskBackendFactoryToStoreNames, jobContext,
+        containerModel, samzaContainerMetrics, checkpointManager, config, taskInstanceMetrics, executor, loggerStoreDir,
+        forceRestoreTasks);
+  }
+
+  /**
+   * Restores all TaskInstances and returns a future for each TaskInstance restore. Note: In case of
+   * {@link BlobStoreRestoreManager}, this method restore with getDeleted flag if it receives a
+   * {@link DeletedException}. This will create a new Checkpoint.
+   */
+  private static CompletableFuture<Map<TaskName, Checkpoint>> restoreAllTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, Map<TaskName, Checkpoint> taskCheckpoints,
+      Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames, JobContext jobContext,
+      ContainerModel containerModel, SamzaContainerMetrics samzaContainerMetrics, CheckpointManager checkpointManager,
+      Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics, ExecutorService executor,
+      File loggedStoreDir, Set<String> forceRestoreTask) {
+
+    Map<TaskName, CompletableFuture<Checkpoint>> newTaskCheckpoints = new ConcurrentHashMap<>();
+    List<Future<Void>> taskRestoreFutures = new ArrayList<>();
+
+    // Submit restore callable for each taskInstance
+    taskRestoreManagers.forEach((taskInstanceName, restoreManagersMap) -> {
+      // Submit for each restore factory
+      restoreManagersMap.forEach((factoryName, taskRestoreManager) -> {
+        long startTime = System.currentTimeMillis();
+        String taskName = taskInstanceName.getTaskName();
+        LOG.info("Starting restore for state for task: {}", taskName);
+
+        CompletableFuture<Void> restoreFuture;
+        if (forceRestoreTask.contains(taskName) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+          // If init was retried with getDeleted, force restore with getDeleted as well, since init only inits the
+          // restoreManager with deleted SnapshotIndex but does not retry to recover the deleted blobs and delegates it
+          // to restore().
+          // Create an empty future that fails immediately with DeletedException to force retry in restore.
+          restoreFuture = new CompletableFuture<>();
+          restoreFuture.completeExceptionally(new SamzaException(new DeletedException()));
+        } else {
+          restoreFuture = taskRestoreManager.restore();
+        }
+
+        CompletableFuture<Void> taskRestoreFuture = restoreFuture.handle((res, ex) -> {
+          updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+
+          if (ex != null) {
+            if (isUnwrappedExceptionDeletedException(ex)) {
+              LOG.warn(
+                  "Received DeletedException during restore for task {}. Reattempting to get deleted blobs",

Review Comment:
   Minor: s/Reattempting .../Attempting to restore again with getDeletedBlobs set to true.
   
   Minor: use same naming convention for the getDeletedBlob flag in the entire PR (comments, params, logs) for grep-ability. (I see get deleted blobs, GetDeletedBlob, getDeletedBlob, getDeleted etc.)



##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManagerUtil.java:
##########
@@ -411,4 +438,292 @@ public static Set<String> getSideInputStoreNames(
     }
     return sideInputStores;
   }
+
+  /**
+   * Inits and Restores all the task stores.
+   * Note: In case of {@link BlobStoreRestoreManager}, this method retries init and restore with getDeleted flag if it
+   * receives a {@link DeletedException}. This will create a new checkpoint for the corresponding task.
+   */
+  public static CompletableFuture<Map<TaskName, Checkpoint>> initAndRestoreTaskInstances(

Review Comment:
   Minor: Suggest moving all of these to a new Util class, e.g.  ContainerStorageManagerRestoreUtil



##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreBackupManager.java:
##########
@@ -139,8 +139,10 @@ public void init(Checkpoint checkpoint) {
 
     // Note: blocks the caller thread.
     // TODO LOW shesharma exclude stores that are no longer configured during init
+    // NOTE: Get SnapshotIndex with getDeleted set to false. A failure to get a blob from SnapshotIndex would restart
+    // the container and the init()/restore() should be able to create a new Snapshot in the blob store and recover.

Review Comment:
   Minor: Mention that this helps with rare race conditions where the snapshot index blob got deleted after the restore compledted successfully. 



##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreRestoreManager.java:
##########
@@ -254,7 +269,10 @@ static CompletableFuture<Void> restoreStores(String jobName, String jobId, TaskN
         throw new SamzaException(String.format("Error deleting store directory: %s", storeDir), e);
       }
 
-      boolean shouldRestore = shouldRestore(taskName.getTaskName(), storeName, dirIndex,
+      // If getDeletedBlobs is enabled - always restore so that we get all the blobs, including the deleted blobs,

Review Comment:
   Minor: Run on sentence. Rephrase for clarity.



##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/util/BlobStoreUtil.java:
##########
@@ -210,6 +203,51 @@ public CompletableFuture<String> putSnapshotIndex(SnapshotIndex snapshotIndex) {
     }, isCauseNonRetriable(), executor, retryPolicyConfig);
   }
 
+  /**
+   * Gets SnapshotIndex blob, cleans up a SnapshotIndex by recursively deleting all blobs associated with files/subdirs
+   * inside the SnapshotIndex and finally deletes SnapshotIndex blob itself.
+   * @param snapshotIndexBlobId Blob id of SnapshotIndex
+   * @param requestMetadata Metadata of the request
+   * @param getDeletedBlob Determines whether to try to get deleted SnapshotIndex or not.
+   */
+  public CompletionStage<Void> cleanSnapshotIndex(String snapshotIndexBlobId, Metadata requestMetadata, boolean getDeletedBlob) {
+    Metadata getSnapshotRequest = new Metadata(Metadata.SNAPSHOT_INDEX_PAYLOAD_PATH, Optional.empty(), requestMetadata.getJobName(),
+        requestMetadata.getJobId(), requestMetadata.getTaskName(), requestMetadata.getStoreName());
+    return getSnapshotIndex(snapshotIndexBlobId, getSnapshotRequest, getDeletedBlob)
+        .thenCompose(snapshotIndex -> cleanSnapshotIndex(snapshotIndexBlobId, snapshotIndex, requestMetadata));
+  }
+
+  /**
+   * Cleans up a SnapshotIndex by recursively deleting all blobs associated with files/subdirs inside the SnapshotIndex
+   * and finally deletes SnapshotIndex blob itself.
+   * @param snapshotIndexBlobId Blob if of SnapshotIndex
+   * @param snapshotIndex SnapshotIndex to delete
+   * @param requestMetadata Metadata of the request
+   */
+  public CompletionStage<Void> cleanSnapshotIndex(String snapshotIndexBlobId, SnapshotIndex snapshotIndex, Metadata requestMetadata) {
+    DirIndex dirIndex = snapshotIndex.getDirIndex();
+    CompletionStage<Void> storeDeletionFuture =
+        cleanUpDir(dirIndex, requestMetadata) // delete files and sub-dirs previously marked for removal
+            .thenComposeAsync(v ->
+                deleteDir(dirIndex, requestMetadata), executor) // deleted files and dirs still present
+            .thenComposeAsync(v -> deleteSnapshotIndexBlob(snapshotIndexBlobId, requestMetadata), executor) // delete the snapshot index blob
+            .exceptionally(ex -> {
+              Throwable unwrappedException = FutureUtil.unwrapExceptions(CompletionException.class,
+                  FutureUtil.unwrapExceptions(SamzaException.class, ex));
+              // If SnapshotIndex is already deleted, do not fail -> this may happen if after we restore a

Review Comment:
   Minor: s/SnapshotIndex/blob, since the error may occur for deleted files as well.



##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/util/BlobStoreUtil.java:
##########
@@ -229,9 +267,10 @@ public CompletionStage<Void> deleteSnapshotIndexBlob(String snapshotIndexBlobId,
   /**
    * Non-blocking restore of a {@link SnapshotIndex} to local store by downloading all the files and sub-dirs associated
    * with this remote snapshot.
+   * NOTE: getDeletedFiles flag sets if it reattempts to get a deleted file by setting getDeleted flag in getFiles.
    * @return A future that completes when all the async downloads completes
    */
-  public CompletableFuture<Void> restoreDir(File baseDir, DirIndex dirIndex, Metadata metadata) {
+  public CompletableFuture<Void> restoreDir(File baseDir, DirIndex dirIndex, Metadata metadata, boolean getDeletedFiles) {
     LOG.debug("Restoring contents of directory: {} from remote snapshot.", baseDir);

Review Comment:
   Minor: Log boolean flag.



##########
samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala:
##########
@@ -993,10 +1000,21 @@ class SamzaContainer(
     })
   }
 
-  def startTask {
+  /**
+   * Init all task instances
+   * @param taskCheckpoints last checkpoint for a TaskName. This last checkpoint could be different from the one returned
+   *                        from CommitManager#getLastCheckpoint. The new checkpoint could be created in case the last

Review Comment:
   s/CommitManager/CheckpointManager or OffsetManager?



##########
samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala:
##########
@@ -149,31 +149,40 @@ class TaskInstance(
     }
   }
 
-  def initTask {
+  def initTask(lastTaskCheckpoint: Option[Checkpoint]) {
     initCaughtUpMapping()
 
+    val isStandByTask = taskModel.getTaskMode == TaskMode.Standby
+    var checkpoint: Checkpoint = lastTaskCheckpoint.orNull
+
     if (commitManager != null) {
       debug("Starting commit manager for taskName: %s" format taskName)
-
-      commitManager.init()
+      if (isStandByTask) {
+        debug("Standby task: %s Passing null to init for taskName %s:" format (isStandByTask, taskName))

Review Comment:
   Minor: taskName: %s (typo). Ideally remove colons for consistency with next message.



##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreBackupManager.java:
##########
@@ -327,7 +329,12 @@ public CompletableFuture<Void> cleanUp(CheckpointId checkpointId, Map<String, St
     });
 
     return FutureUtil.allOf(removeTTLFutures, cleanupRemoteSnapshotFutures, removePrevRemoteSnapshotFutures)
-        .whenComplete((res, ex) -> metrics.cleanupNs.update(System.nanoTime() - startTime));
+        .whenComplete((res, ex) -> {
+          if (ex != null) {
+            LOG.error("Could not finish cleanup. Checkpoint id: {}, store SCMs: {}", checkpointId, storeSCMs, ex);

Review Comment:
   Minor: "checkpointId" for consistency with other logs and grep-ability.



##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/util/BlobStoreUtil.java:
##########
@@ -424,7 +464,7 @@ CompletableFuture<Void> getFile(List<FileBlob> fileBlobs, File fileToRestore, Me
         resultFuture = resultFuture.thenComposeAsync(v -> {
           LOG.debug("Starting restore for file: {} with blob id: {} at offset: {}", fileToRestore, fileBlob.getBlobId(),

Review Comment:
   Minor: Log boolean flag too.



##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java:
##########
@@ -301,79 +303,28 @@ private void restoreStores() throws InterruptedException {
               samzaContainerMetrics, taskInstanceMetrics, taskInstanceCollectors, serdes,
               loggedStoreBaseDirectory, nonLoggedStoreBaseDirectory, config, clock);
       taskRestoreManagers.put(taskName, taskStoreRestoreManagers);
+      taskBackendFactoryToStoreNames.put(taskName, backendFactoryToStoreNames);
     });
 
-    // Initialize each TaskStorageManager
-    taskRestoreManagers.forEach((taskName, restoreManagers) ->
-        restoreManagers.forEach((factoryName, taskRestoreManager) ->
-            taskRestoreManager.init(taskCheckpoints.get(taskName))
-        )
-    );
-
-    // Start each store consumer once.
-    // Note: These consumers are per system and only changelog system store consumers will be started.
-    // Some TaskRestoreManagers may not require the consumer to to be started, but due to the agnostic nature of
-    // ContainerStorageManager we always start the changelog consumer here in case it is required
-    this.storeConsumers.values().stream().distinct().forEach(SystemConsumer::start);
-
-    List<Future<Void>> taskRestoreFutures = new ArrayList<>();
-
-    // Submit restore callable for each taskInstance
-    taskRestoreManagers.forEach((taskInstance, restoreManagersMap) -> {
-      // Submit for each restore factory
-      restoreManagersMap.forEach((factoryName, taskRestoreManager) -> {
-        long startTime = System.currentTimeMillis();
-        String taskName = taskInstance.getTaskName();
-        LOG.info("Starting restore for state for task: {}", taskName);
-        CompletableFuture<Void> restoreFuture = taskRestoreManager.restore().handle((res, ex) -> {
-          // Stop all persistent stores after restoring. Certain persistent stores opened in BulkLoad mode are compacted
-          // on stop, so paralleling stop() also parallelizes their compaction (a time-intensive operation).
-          try {
-            taskRestoreManager.close();
-          } catch (Exception e) {
-            LOG.error("Error closing restore manager for task: {} after {} restore",
-                taskName, ex != null ? "unsuccessful" : "successful", e);
-            // ignore exception from close. container may still be be able to continue processing/backups
-            // if restore manager close fails.
-          }
-
-          long timeToRestore = System.currentTimeMillis() - startTime;
-          if (samzaContainerMetrics != null) {
-            Gauge taskGauge = samzaContainerMetrics.taskStoreRestorationMetrics().getOrDefault(taskInstance, null);
-
-            if (taskGauge != null) {
-              taskGauge.set(timeToRestore);
-            }
-          }
-
-          if (ex != null) {
-            // log and rethrow exception to communicate restore failure
-            String msg = String.format("Error restoring state for task: %s", taskName);
-            LOG.error(msg, ex);
-            throw new SamzaException(msg, ex); // wrap in unchecked exception to throw from lambda
-          } else {
-            return null;
-          }
-        });
-
-        taskRestoreFutures.add(restoreFuture);
-      });
-    });
+    // Init all taskRestores and if successful, restores all the task stores concurrently
+    CompletableFuture<Map<TaskName, Checkpoint>> initRestoreAndNewCheckpointFuture =
+        ContainerStorageManagerUtil.initAndRestoreTaskInstances(taskRestoreManagers, samzaContainerMetrics,
+            checkpointManager, jobContext, containerModel, taskCheckpoints, taskBackendFactoryToStoreNames, config,
+            restoreExecutor, taskInstanceMetrics, loggedStoreBaseDirectory, storeConsumers);
 
-    // Loop-over the future list to wait for each restore to finish, catch any exceptions during restore and throw
-    // as samza exceptions
-    for (Future<Void> future : taskRestoreFutures) {
-      try {
-        future.get();
-      } catch (InterruptedException e) {
-        LOG.warn("Received an interrupt during store restoration. Interrupting the restore executor to exit "
-            + "prematurely without restoring full state.");
-        restoreExecutor.shutdownNow();
-        throw e;
-      } catch (Exception e) {
-        LOG.error("Exception when restoring state.", e);
-        throw new SamzaException("Exception when restoring state.", e);
-      }
+    // Update the task checkpoints map, if it was updated during the restore. Throw an exception if the restore or
+    // creating a new checkpoint (in case of BlobStoreBackendFactory) failed.
+    try {
+      Map<TaskName, Checkpoint> newTaskCheckpoints = initRestoreAndNewCheckpointFuture.get();
+      taskCheckpoints.putAll(newTaskCheckpoints);

Review Comment:
   Minor: Can return the newTaskCheckpoints map directly instead of copying over.



##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManagerUtil.java:
##########
@@ -411,4 +438,292 @@ public static Set<String> getSideInputStoreNames(
     }
     return sideInputStores;
   }
+
+  /**
+   * Inits and Restores all the task stores.
+   * Note: In case of {@link BlobStoreRestoreManager}, this method retries init and restore with getDeleted flag if it
+   * receives a {@link DeletedException}. This will create a new checkpoint for the corresponding task.
+   */
+  public static CompletableFuture<Map<TaskName, Checkpoint>> initAndRestoreTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, SamzaContainerMetrics samzaContainerMetrics,
+      CheckpointManager checkpointManager, JobContext jobContext, ContainerModel containerModel,
+      Map<TaskName, Checkpoint> taskCheckpoints, Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames,
+      Config config, ExecutorService executor, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      File loggerStoreDir, Map<String, SystemConsumer> storeConsumers) {
+
+    Set<String> forceRestoreTasks = new HashSet<>();
+    // Initialize each TaskStorageManager.
+    taskRestoreManagers.forEach((taskName, restoreManagers) ->
+        restoreManagers.forEach((factoryName, taskRestoreManager) -> {
+          try {
+            taskRestoreManager.init(taskCheckpoints.get(taskName));
+          } catch (SamzaException ex) {
+            if (isUnwrappedExceptionDeletedException(ex) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+              // Get deleted SnapshotIndex blob with GetDeleted and mark the task to be restored with GetDeleted as well.
+              // this ensures that the restore downloads the snapshot, recreates a new snapshot, uploads it to blob store
+              // and creates a new checkpoint.
+              ((BlobStoreRestoreManager) taskRestoreManager).init(taskCheckpoints.get(taskName), true);
+              forceRestoreTasks.add(taskName.getTaskName());
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("init failed for BlobStoreRestoreManager, task: %s with GetDeleted set to true", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex);
+            }
+          }
+        })
+    );
+
+    // Start each store consumer once.
+    // Note: These consumers are per system and only changelog system store consumers will be started.
+    // Some TaskRestoreManagers may not require the consumer to be started, but due to the agnostic nature of
+    // ContainerStorageManager we always start the changelog consumer here in case it is required
+    storeConsumers.values().stream().distinct().forEach(SystemConsumer::start);
+
+    return restoreAllTaskInstances(taskRestoreManagers, taskCheckpoints, taskBackendFactoryToStoreNames, jobContext,
+        containerModel, samzaContainerMetrics, checkpointManager, config, taskInstanceMetrics, executor, loggerStoreDir,
+        forceRestoreTasks);
+  }
+
+  /**
+   * Restores all TaskInstances and returns a future for each TaskInstance restore. Note: In case of
+   * {@link BlobStoreRestoreManager}, this method restore with getDeleted flag if it receives a
+   * {@link DeletedException}. This will create a new Checkpoint.
+   */
+  private static CompletableFuture<Map<TaskName, Checkpoint>> restoreAllTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, Map<TaskName, Checkpoint> taskCheckpoints,
+      Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames, JobContext jobContext,
+      ContainerModel containerModel, SamzaContainerMetrics samzaContainerMetrics, CheckpointManager checkpointManager,
+      Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics, ExecutorService executor,
+      File loggedStoreDir, Set<String> forceRestoreTask) {
+
+    Map<TaskName, CompletableFuture<Checkpoint>> newTaskCheckpoints = new ConcurrentHashMap<>();
+    List<Future<Void>> taskRestoreFutures = new ArrayList<>();
+
+    // Submit restore callable for each taskInstance
+    taskRestoreManagers.forEach((taskInstanceName, restoreManagersMap) -> {
+      // Submit for each restore factory
+      restoreManagersMap.forEach((factoryName, taskRestoreManager) -> {
+        long startTime = System.currentTimeMillis();
+        String taskName = taskInstanceName.getTaskName();
+        LOG.info("Starting restore for state for task: {}", taskName);
+
+        CompletableFuture<Void> restoreFuture;
+        if (forceRestoreTask.contains(taskName) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+          // If init was retried with getDeleted, force restore with getDeleted as well, since init only inits the
+          // restoreManager with deleted SnapshotIndex but does not retry to recover the deleted blobs and delegates it
+          // to restore().
+          // Create an empty future that fails immediately with DeletedException to force retry in restore.
+          restoreFuture = new CompletableFuture<>();
+          restoreFuture.completeExceptionally(new SamzaException(new DeletedException()));
+        } else {
+          restoreFuture = taskRestoreManager.restore();
+        }
+
+        CompletableFuture<Void> taskRestoreFuture = restoreFuture.handle((res, ex) -> {
+          updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+
+          if (ex != null) {
+            if (isUnwrappedExceptionDeletedException(ex)) {
+              LOG.warn(
+                  "Received DeletedException during restore for task {}. Reattempting to get deleted blobs",
+                  taskInstanceName.getTaskName());
+
+              // Try to restore with getDeleted flag
+              CompletableFuture<Checkpoint> future =
+                  restoreDeletedSnapshot(taskInstanceName, taskCheckpoints,
+                      checkpointManager, taskRestoreManager, config, taskInstanceMetrics, executor,
+                      taskBackendFactoryToStoreNames.get(taskInstanceName).get(factoryName), loggedStoreDir,
+                      jobContext, containerModel);
+              try {
+                if (future != null) {

Review Comment:
   Minor: Can this be null? If not, simplify.



##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManagerUtil.java:
##########
@@ -411,4 +438,292 @@ public static Set<String> getSideInputStoreNames(
     }
     return sideInputStores;
   }
+
+  /**
+   * Inits and Restores all the task stores.
+   * Note: In case of {@link BlobStoreRestoreManager}, this method retries init and restore with getDeleted flag if it
+   * receives a {@link DeletedException}. This will create a new checkpoint for the corresponding task.
+   */
+  public static CompletableFuture<Map<TaskName, Checkpoint>> initAndRestoreTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, SamzaContainerMetrics samzaContainerMetrics,
+      CheckpointManager checkpointManager, JobContext jobContext, ContainerModel containerModel,
+      Map<TaskName, Checkpoint> taskCheckpoints, Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames,
+      Config config, ExecutorService executor, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      File loggerStoreDir, Map<String, SystemConsumer> storeConsumers) {
+
+    Set<String> forceRestoreTasks = new HashSet<>();
+    // Initialize each TaskStorageManager.
+    taskRestoreManagers.forEach((taskName, restoreManagers) ->
+        restoreManagers.forEach((factoryName, taskRestoreManager) -> {
+          try {
+            taskRestoreManager.init(taskCheckpoints.get(taskName));
+          } catch (SamzaException ex) {
+            if (isUnwrappedExceptionDeletedException(ex) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+              // Get deleted SnapshotIndex blob with GetDeleted and mark the task to be restored with GetDeleted as well.
+              // this ensures that the restore downloads the snapshot, recreates a new snapshot, uploads it to blob store
+              // and creates a new checkpoint.
+              ((BlobStoreRestoreManager) taskRestoreManager).init(taskCheckpoints.get(taskName), true);
+              forceRestoreTasks.add(taskName.getTaskName());
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("init failed for BlobStoreRestoreManager, task: %s with GetDeleted set to true", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex);
+            }
+          }
+        })
+    );
+
+    // Start each store consumer once.
+    // Note: These consumers are per system and only changelog system store consumers will be started.
+    // Some TaskRestoreManagers may not require the consumer to be started, but due to the agnostic nature of
+    // ContainerStorageManager we always start the changelog consumer here in case it is required
+    storeConsumers.values().stream().distinct().forEach(SystemConsumer::start);
+
+    return restoreAllTaskInstances(taskRestoreManagers, taskCheckpoints, taskBackendFactoryToStoreNames, jobContext,
+        containerModel, samzaContainerMetrics, checkpointManager, config, taskInstanceMetrics, executor, loggerStoreDir,
+        forceRestoreTasks);
+  }
+
+  /**
+   * Restores all TaskInstances and returns a future for each TaskInstance restore. Note: In case of
+   * {@link BlobStoreRestoreManager}, this method restore with getDeleted flag if it receives a
+   * {@link DeletedException}. This will create a new Checkpoint.
+   */
+  private static CompletableFuture<Map<TaskName, Checkpoint>> restoreAllTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, Map<TaskName, Checkpoint> taskCheckpoints,
+      Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames, JobContext jobContext,
+      ContainerModel containerModel, SamzaContainerMetrics samzaContainerMetrics, CheckpointManager checkpointManager,
+      Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics, ExecutorService executor,
+      File loggedStoreDir, Set<String> forceRestoreTask) {
+
+    Map<TaskName, CompletableFuture<Checkpoint>> newTaskCheckpoints = new ConcurrentHashMap<>();
+    List<Future<Void>> taskRestoreFutures = new ArrayList<>();
+
+    // Submit restore callable for each taskInstance
+    taskRestoreManagers.forEach((taskInstanceName, restoreManagersMap) -> {
+      // Submit for each restore factory
+      restoreManagersMap.forEach((factoryName, taskRestoreManager) -> {
+        long startTime = System.currentTimeMillis();
+        String taskName = taskInstanceName.getTaskName();
+        LOG.info("Starting restore for state for task: {}", taskName);
+
+        CompletableFuture<Void> restoreFuture;
+        if (forceRestoreTask.contains(taskName) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+          // If init was retried with getDeleted, force restore with getDeleted as well, since init only inits the
+          // restoreManager with deleted SnapshotIndex but does not retry to recover the deleted blobs and delegates it
+          // to restore().
+          // Create an empty future that fails immediately with DeletedException to force retry in restore.
+          restoreFuture = new CompletableFuture<>();
+          restoreFuture.completeExceptionally(new SamzaException(new DeletedException()));
+        } else {
+          restoreFuture = taskRestoreManager.restore();
+        }
+
+        CompletableFuture<Void> taskRestoreFuture = restoreFuture.handle((res, ex) -> {
+          updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+
+          if (ex != null) {
+            if (isUnwrappedExceptionDeletedException(ex)) {
+              LOG.warn(
+                  "Received DeletedException during restore for task {}. Reattempting to get deleted blobs",
+                  taskInstanceName.getTaskName());
+
+              // Try to restore with getDeleted flag
+              CompletableFuture<Checkpoint> future =
+                  restoreDeletedSnapshot(taskInstanceName, taskCheckpoints,
+                      checkpointManager, taskRestoreManager, config, taskInstanceMetrics, executor,
+                      taskBackendFactoryToStoreNames.get(taskInstanceName).get(factoryName), loggedStoreDir,
+                      jobContext, containerModel);
+              try {
+                if (future != null) {
+                  newTaskCheckpoints.put(taskInstanceName, future);
+                }
+              } catch (Exception e) {
+                String msg =

Review Comment:
   Minor: This should say "Error restoring deleted snapshot with getDeletedBlobs set to true" or something. We don't know it's a DeletedException



##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManagerUtil.java:
##########
@@ -411,4 +438,292 @@ public static Set<String> getSideInputStoreNames(
     }
     return sideInputStores;
   }
+
+  /**
+   * Inits and Restores all the task stores.
+   * Note: In case of {@link BlobStoreRestoreManager}, this method retries init and restore with getDeleted flag if it
+   * receives a {@link DeletedException}. This will create a new checkpoint for the corresponding task.
+   */
+  public static CompletableFuture<Map<TaskName, Checkpoint>> initAndRestoreTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, SamzaContainerMetrics samzaContainerMetrics,
+      CheckpointManager checkpointManager, JobContext jobContext, ContainerModel containerModel,
+      Map<TaskName, Checkpoint> taskCheckpoints, Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames,
+      Config config, ExecutorService executor, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      File loggerStoreDir, Map<String, SystemConsumer> storeConsumers) {
+
+    Set<String> forceRestoreTasks = new HashSet<>();
+    // Initialize each TaskStorageManager.
+    taskRestoreManagers.forEach((taskName, restoreManagers) ->
+        restoreManagers.forEach((factoryName, taskRestoreManager) -> {
+          try {
+            taskRestoreManager.init(taskCheckpoints.get(taskName));
+          } catch (SamzaException ex) {
+            if (isUnwrappedExceptionDeletedException(ex) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+              // Get deleted SnapshotIndex blob with GetDeleted and mark the task to be restored with GetDeleted as well.
+              // this ensures that the restore downloads the snapshot, recreates a new snapshot, uploads it to blob store
+              // and creates a new checkpoint.
+              ((BlobStoreRestoreManager) taskRestoreManager).init(taskCheckpoints.get(taskName), true);
+              forceRestoreTasks.add(taskName.getTaskName());
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("init failed for BlobStoreRestoreManager, task: %s with GetDeleted set to true", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex);
+            }
+          }
+        })
+    );
+
+    // Start each store consumer once.
+    // Note: These consumers are per system and only changelog system store consumers will be started.
+    // Some TaskRestoreManagers may not require the consumer to be started, but due to the agnostic nature of
+    // ContainerStorageManager we always start the changelog consumer here in case it is required
+    storeConsumers.values().stream().distinct().forEach(SystemConsumer::start);
+
+    return restoreAllTaskInstances(taskRestoreManagers, taskCheckpoints, taskBackendFactoryToStoreNames, jobContext,
+        containerModel, samzaContainerMetrics, checkpointManager, config, taskInstanceMetrics, executor, loggerStoreDir,
+        forceRestoreTasks);
+  }
+
+  /**
+   * Restores all TaskInstances and returns a future for each TaskInstance restore. Note: In case of
+   * {@link BlobStoreRestoreManager}, this method restore with getDeleted flag if it receives a
+   * {@link DeletedException}. This will create a new Checkpoint.
+   */
+  private static CompletableFuture<Map<TaskName, Checkpoint>> restoreAllTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, Map<TaskName, Checkpoint> taskCheckpoints,
+      Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames, JobContext jobContext,
+      ContainerModel containerModel, SamzaContainerMetrics samzaContainerMetrics, CheckpointManager checkpointManager,
+      Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics, ExecutorService executor,
+      File loggedStoreDir, Set<String> forceRestoreTask) {
+
+    Map<TaskName, CompletableFuture<Checkpoint>> newTaskCheckpoints = new ConcurrentHashMap<>();
+    List<Future<Void>> taskRestoreFutures = new ArrayList<>();
+
+    // Submit restore callable for each taskInstance
+    taskRestoreManagers.forEach((taskInstanceName, restoreManagersMap) -> {
+      // Submit for each restore factory
+      restoreManagersMap.forEach((factoryName, taskRestoreManager) -> {
+        long startTime = System.currentTimeMillis();
+        String taskName = taskInstanceName.getTaskName();
+        LOG.info("Starting restore for state for task: {}", taskName);
+
+        CompletableFuture<Void> restoreFuture;
+        if (forceRestoreTask.contains(taskName) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+          // If init was retried with getDeleted, force restore with getDeleted as well, since init only inits the
+          // restoreManager with deleted SnapshotIndex but does not retry to recover the deleted blobs and delegates it
+          // to restore().
+          // Create an empty future that fails immediately with DeletedException to force retry in restore.
+          restoreFuture = new CompletableFuture<>();
+          restoreFuture.completeExceptionally(new SamzaException(new DeletedException()));
+        } else {
+          restoreFuture = taskRestoreManager.restore();
+        }
+
+        CompletableFuture<Void> taskRestoreFuture = restoreFuture.handle((res, ex) -> {
+          updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+
+          if (ex != null) {
+            if (isUnwrappedExceptionDeletedException(ex)) {
+              LOG.warn(
+                  "Received DeletedException during restore for task {}. Reattempting to get deleted blobs",
+                  taskInstanceName.getTaskName());
+
+              // Try to restore with getDeleted flag
+              CompletableFuture<Checkpoint> future =
+                  restoreDeletedSnapshot(taskInstanceName, taskCheckpoints,
+                      checkpointManager, taskRestoreManager, config, taskInstanceMetrics, executor,
+                      taskBackendFactoryToStoreNames.get(taskInstanceName).get(factoryName), loggedStoreDir,
+                      jobContext, containerModel);
+              try {
+                if (future != null) {
+                  newTaskCheckpoints.put(taskInstanceName, future);
+                }
+              } catch (Exception e) {

Review Comment:
   Minor: Should the restoreDeletedSnapshot method call be in try-catch too to capture any synchronous exceptions?



##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManagerUtil.java:
##########
@@ -411,4 +438,292 @@ public static Set<String> getSideInputStoreNames(
     }
     return sideInputStores;
   }
+
+  /**
+   * Inits and Restores all the task stores.
+   * Note: In case of {@link BlobStoreRestoreManager}, this method retries init and restore with getDeleted flag if it
+   * receives a {@link DeletedException}. This will create a new checkpoint for the corresponding task.
+   */
+  public static CompletableFuture<Map<TaskName, Checkpoint>> initAndRestoreTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, SamzaContainerMetrics samzaContainerMetrics,
+      CheckpointManager checkpointManager, JobContext jobContext, ContainerModel containerModel,
+      Map<TaskName, Checkpoint> taskCheckpoints, Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames,
+      Config config, ExecutorService executor, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      File loggerStoreDir, Map<String, SystemConsumer> storeConsumers) {
+
+    Set<String> forceRestoreTasks = new HashSet<>();
+    // Initialize each TaskStorageManager.
+    taskRestoreManagers.forEach((taskName, restoreManagers) ->
+        restoreManagers.forEach((factoryName, taskRestoreManager) -> {
+          try {
+            taskRestoreManager.init(taskCheckpoints.get(taskName));
+          } catch (SamzaException ex) {
+            if (isUnwrappedExceptionDeletedException(ex) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+              // Get deleted SnapshotIndex blob with GetDeleted and mark the task to be restored with GetDeleted as well.
+              // this ensures that the restore downloads the snapshot, recreates a new snapshot, uploads it to blob store
+              // and creates a new checkpoint.
+              ((BlobStoreRestoreManager) taskRestoreManager).init(taskCheckpoints.get(taskName), true);
+              forceRestoreTasks.add(taskName.getTaskName());
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("init failed for BlobStoreRestoreManager, task: %s with GetDeleted set to true", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex);
+            }
+          }
+        })
+    );
+
+    // Start each store consumer once.
+    // Note: These consumers are per system and only changelog system store consumers will be started.
+    // Some TaskRestoreManagers may not require the consumer to be started, but due to the agnostic nature of
+    // ContainerStorageManager we always start the changelog consumer here in case it is required
+    storeConsumers.values().stream().distinct().forEach(SystemConsumer::start);
+
+    return restoreAllTaskInstances(taskRestoreManagers, taskCheckpoints, taskBackendFactoryToStoreNames, jobContext,
+        containerModel, samzaContainerMetrics, checkpointManager, config, taskInstanceMetrics, executor, loggerStoreDir,
+        forceRestoreTasks);
+  }
+
+  /**
+   * Restores all TaskInstances and returns a future for each TaskInstance restore. Note: In case of
+   * {@link BlobStoreRestoreManager}, this method restore with getDeleted flag if it receives a
+   * {@link DeletedException}. This will create a new Checkpoint.
+   */
+  private static CompletableFuture<Map<TaskName, Checkpoint>> restoreAllTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, Map<TaskName, Checkpoint> taskCheckpoints,
+      Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames, JobContext jobContext,
+      ContainerModel containerModel, SamzaContainerMetrics samzaContainerMetrics, CheckpointManager checkpointManager,
+      Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics, ExecutorService executor,
+      File loggedStoreDir, Set<String> forceRestoreTask) {
+
+    Map<TaskName, CompletableFuture<Checkpoint>> newTaskCheckpoints = new ConcurrentHashMap<>();
+    List<Future<Void>> taskRestoreFutures = new ArrayList<>();
+
+    // Submit restore callable for each taskInstance
+    taskRestoreManagers.forEach((taskInstanceName, restoreManagersMap) -> {
+      // Submit for each restore factory
+      restoreManagersMap.forEach((factoryName, taskRestoreManager) -> {
+        long startTime = System.currentTimeMillis();
+        String taskName = taskInstanceName.getTaskName();
+        LOG.info("Starting restore for state for task: {}", taskName);
+
+        CompletableFuture<Void> restoreFuture;
+        if (forceRestoreTask.contains(taskName) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+          // If init was retried with getDeleted, force restore with getDeleted as well, since init only inits the
+          // restoreManager with deleted SnapshotIndex but does not retry to recover the deleted blobs and delegates it
+          // to restore().
+          // Create an empty future that fails immediately with DeletedException to force retry in restore.
+          restoreFuture = new CompletableFuture<>();
+          restoreFuture.completeExceptionally(new SamzaException(new DeletedException()));
+        } else {
+          restoreFuture = taskRestoreManager.restore();
+        }
+
+        CompletableFuture<Void> taskRestoreFuture = restoreFuture.handle((res, ex) -> {
+          updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+
+          if (ex != null) {
+            if (isUnwrappedExceptionDeletedException(ex)) {
+              LOG.warn(
+                  "Received DeletedException during restore for task {}. Reattempting to get deleted blobs",
+                  taskInstanceName.getTaskName());
+
+              // Try to restore with getDeleted flag
+              CompletableFuture<Checkpoint> future =
+                  restoreDeletedSnapshot(taskInstanceName, taskCheckpoints,
+                      checkpointManager, taskRestoreManager, config, taskInstanceMetrics, executor,
+                      taskBackendFactoryToStoreNames.get(taskInstanceName).get(factoryName), loggedStoreDir,
+                      jobContext, containerModel);
+              try {
+                if (future != null) {
+                  newTaskCheckpoints.put(taskInstanceName, future);
+                }
+              } catch (Exception e) {
+                String msg =
+                    String.format("DeletedException during restore task: %s after retrying to get deleted blobs.", taskName);
+                throw new SamzaException(msg, e);
+              } finally {
+                updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+              }
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("Error restoring state for task: %s", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex); // wrap in unchecked exception to throw from lambda
+            }
+          }
+
+          // Stop all persistent stores after restoring. Certain persistent stores opened in BulkLoad mode are compacted
+          // on stop, so paralleling stop() also parallelizes their compaction (a time-intensive operation).
+          try {
+            taskRestoreManager.close();
+          } catch (Exception e) {
+            LOG.error("Error closing restore manager for task: {} after {} restore", taskName,
+                ex != null ? "unsuccessful" : "successful", e);
+            // ignore exception from close. container may still be able to continue processing/backups
+            // if restore manager close fails.
+          }
+          return null;
+        });
+        taskRestoreFutures.add(taskRestoreFuture);
+      });
+    });
+    CompletableFuture<Void> restoreFutures = CompletableFuture.allOf(taskRestoreFutures.toArray(new CompletableFuture[0]));
+    return restoreFutures.thenCompose(ignoredVoid -> FutureUtil.toFutureOfMap(newTaskCheckpoints));
+  }
+
+  /**
+   * Returns a single future that guarantees all the following are completed, in this order:
+   *   1. Restore state locally by getting deleted blobs from the blob store.
+   *   2. Create a new snapshot from restored state by backing it up on the blob store.
+   *   3. Remove TTL from the new Snapshot and all the associated blobs in the blob store
+   *   4. Clean up old/deleted Snapshot
+   *   5. Create and write the new checkpoint to checkpoint topic
+   */
+  private static CompletableFuture<Checkpoint> restoreDeletedSnapshot(TaskName taskName,
+      Map<TaskName, Checkpoint> taskCheckpoints, CheckpointManager checkpointManager,
+      TaskRestoreManager taskRestoreManager, Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      ExecutorService executor, Set<String> storesToRestore, File loggedStoreBaseDirectory, JobContext jobContext,
+      ContainerModel containerModel) {
+
+    // if taskInstanceMetrics are specified use those for store metrics,
+    // otherwise (in case of StorageRecovery) use a blank MetricsRegistryMap
+    MetricsRegistry metricsRegistry =
+        taskInstanceMetrics.get(taskName) != null ? taskInstanceMetrics.get(taskName).registry()
+            : new MetricsRegistryMap();
+
+    BlobStoreManager blobStoreManager = getBlobStoreManager(config, executor);
+    JobConfig jobConfig = new JobConfig(config);
+    BlobStoreUtil blobStoreUtil =
+        new BlobStoreUtil(blobStoreManager, executor, new BlobStoreConfig(config), null,
+            new BlobStoreRestoreManagerMetrics(metricsRegistry));
+
+    BlobStoreRestoreManager blobStoreRestoreManager = (BlobStoreRestoreManager) taskRestoreManager;
+
+    CheckpointId checkpointId = CheckpointId.create();
+    Map<String, String> oldSCMs = ((CheckpointV2) taskCheckpoints.get(taskName)).getStateCheckpointMarkers()
+        .get(BlobStoreStateBackendFactory.class.getName());
+
+    // 1. Restore state with getDeleted flag set to true
+    CompletableFuture<Void> restoreFuture = blobStoreRestoreManager.restore(true);
+
+    // 2. Create a new checkpoint and back it up on the blob store
+    CompletableFuture<Map<String, String>> backupStoresFuture = restoreFuture.thenCompose(
+      r -> backupRecoveredStore(jobContext, containerModel, config, taskName, storesToRestore, checkpointId,
+          loggedStoreBaseDirectory, blobStoreManager, metricsRegistry, executor));
+
+    // 3. Mark new Snapshots to never expire
+    CompletableFuture<Void> removeNewSnapshotsTTLFuture = backupStoresFuture.thenCompose(
+      storeSCMs -> {
+        List<CompletableFuture<Void>> removeSnapshotTTLFutures = new ArrayList<>();
+        storeSCMs.forEach((store, scm) -> {
+          Metadata requestMetadata = new Metadata(Metadata.SNAPSHOT_INDEX_PAYLOAD_PATH, Optional.empty(),
+              jobConfig.getName().get(), jobConfig.getJobId(), taskName.getTaskName(), store);
+          removeSnapshotTTLFutures.add(blobStoreUtil.getSnapshotIndexAndRemoveTTL(scm, requestMetadata));

Review Comment:
   Minor: Should rename util method to "removeTTLForSnapshotIndex" (use same naming convention as regular cleanup co clarify whether "for SnapshotIndex" includes its files or not.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] shekhars-li commented on a diff in pull request #1676: [SAMZA-2787] GetDeleted API and Recover from DeletedException

Posted by "shekhars-li (via GitHub)" <gi...@apache.org>.
shekhars-li commented on code in PR #1676:
URL: https://github.com/apache/samza/pull/1676#discussion_r1289227259


##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManagerUtil.java:
##########
@@ -411,4 +438,292 @@ public static Set<String> getSideInputStoreNames(
     }
     return sideInputStores;
   }
+
+  /**
+   * Inits and Restores all the task stores.
+   * Note: In case of {@link BlobStoreRestoreManager}, this method retries init and restore with getDeleted flag if it
+   * receives a {@link DeletedException}. This will create a new checkpoint for the corresponding task.
+   */
+  public static CompletableFuture<Map<TaskName, Checkpoint>> initAndRestoreTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, SamzaContainerMetrics samzaContainerMetrics,
+      CheckpointManager checkpointManager, JobContext jobContext, ContainerModel containerModel,
+      Map<TaskName, Checkpoint> taskCheckpoints, Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames,
+      Config config, ExecutorService executor, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      File loggerStoreDir, Map<String, SystemConsumer> storeConsumers) {
+
+    Set<String> forceRestoreTasks = new HashSet<>();
+    // Initialize each TaskStorageManager.
+    taskRestoreManagers.forEach((taskName, restoreManagers) ->
+        restoreManagers.forEach((factoryName, taskRestoreManager) -> {
+          try {
+            taskRestoreManager.init(taskCheckpoints.get(taskName));
+          } catch (SamzaException ex) {
+            if (isUnwrappedExceptionDeletedException(ex) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+              // Get deleted SnapshotIndex blob with GetDeleted and mark the task to be restored with GetDeleted as well.
+              // this ensures that the restore downloads the snapshot, recreates a new snapshot, uploads it to blob store
+              // and creates a new checkpoint.
+              ((BlobStoreRestoreManager) taskRestoreManager).init(taskCheckpoints.get(taskName), true);
+              forceRestoreTasks.add(taskName.getTaskName());
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("init failed for BlobStoreRestoreManager, task: %s with GetDeleted set to true", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex);
+            }
+          }
+        })
+    );
+
+    // Start each store consumer once.
+    // Note: These consumers are per system and only changelog system store consumers will be started.
+    // Some TaskRestoreManagers may not require the consumer to be started, but due to the agnostic nature of
+    // ContainerStorageManager we always start the changelog consumer here in case it is required
+    storeConsumers.values().stream().distinct().forEach(SystemConsumer::start);
+
+    return restoreAllTaskInstances(taskRestoreManagers, taskCheckpoints, taskBackendFactoryToStoreNames, jobContext,
+        containerModel, samzaContainerMetrics, checkpointManager, config, taskInstanceMetrics, executor, loggerStoreDir,
+        forceRestoreTasks);
+  }
+
+  /**
+   * Restores all TaskInstances and returns a future for each TaskInstance restore. Note: In case of
+   * {@link BlobStoreRestoreManager}, this method restore with getDeleted flag if it receives a
+   * {@link DeletedException}. This will create a new Checkpoint.
+   */
+  private static CompletableFuture<Map<TaskName, Checkpoint>> restoreAllTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, Map<TaskName, Checkpoint> taskCheckpoints,
+      Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames, JobContext jobContext,
+      ContainerModel containerModel, SamzaContainerMetrics samzaContainerMetrics, CheckpointManager checkpointManager,
+      Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics, ExecutorService executor,
+      File loggedStoreDir, Set<String> forceRestoreTask) {
+
+    Map<TaskName, CompletableFuture<Checkpoint>> newTaskCheckpoints = new ConcurrentHashMap<>();
+    List<Future<Void>> taskRestoreFutures = new ArrayList<>();
+
+    // Submit restore callable for each taskInstance
+    taskRestoreManagers.forEach((taskInstanceName, restoreManagersMap) -> {
+      // Submit for each restore factory
+      restoreManagersMap.forEach((factoryName, taskRestoreManager) -> {
+        long startTime = System.currentTimeMillis();
+        String taskName = taskInstanceName.getTaskName();
+        LOG.info("Starting restore for state for task: {}", taskName);
+
+        CompletableFuture<Void> restoreFuture;
+        if (forceRestoreTask.contains(taskName) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+          // If init was retried with getDeleted, force restore with getDeleted as well, since init only inits the
+          // restoreManager with deleted SnapshotIndex but does not retry to recover the deleted blobs and delegates it
+          // to restore().
+          // Create an empty future that fails immediately with DeletedException to force retry in restore.
+          restoreFuture = new CompletableFuture<>();
+          restoreFuture.completeExceptionally(new SamzaException(new DeletedException()));
+        } else {
+          restoreFuture = taskRestoreManager.restore();
+        }
+
+        CompletableFuture<Void> taskRestoreFuture = restoreFuture.handle((res, ex) -> {
+          updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+
+          if (ex != null) {
+            if (isUnwrappedExceptionDeletedException(ex)) {
+              LOG.warn(
+                  "Received DeletedException during restore for task {}. Reattempting to get deleted blobs",

Review Comment:
   Updated all the occurrence in logs/params etc to getDeleted. Also added it to more logs for better traceability. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] shekhars-li commented on a diff in pull request #1676: [SAMZA-2787] GetDeleted API and Recover from DeletedException

Posted by "shekhars-li (via GitHub)" <gi...@apache.org>.
shekhars-li commented on code in PR #1676:
URL: https://github.com/apache/samza/pull/1676#discussion_r1286263098


##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreRestoreManager.java:
##########
@@ -153,6 +158,10 @@ public CompletableFuture<Void> restore() {
         storageConfig, metrics, storageManagerUtil, blobStoreUtil, dirDiffUtil, executor);
   }
 
+  public CompletableFuture<Void> restore(Boolean restoreDeleted) {

Review Comment:
   Same as init -> implemented a version from interface without getDeleted and one with getDeleted for our purpose. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] shekhars-li commented on a diff in pull request #1676: [SAMZA-2787] GetDeleted API and Recover from DeletedException

Posted by "shekhars-li (via GitHub)" <gi...@apache.org>.
shekhars-li commented on code in PR #1676:
URL: https://github.com/apache/samza/pull/1676#discussion_r1287943905


##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreRestoreManager.java:
##########
@@ -254,8 +263,10 @@ static CompletableFuture<Void> restoreStores(String jobName, String jobId, TaskN
         throw new SamzaException(String.format("Error deleting store directory: %s", storeDir), e);
       }
 
+      // If getDeletedBlobs is enabled - always restore so that we get all the blobs, including the deleted blobs,
+      // immediately restore it locally and backup to create new checkpoint.
       boolean shouldRestore = shouldRestore(taskName.getTaskName(), storeName, dirIndex,
-          storeCheckpointDir, storageConfig, dirDiffUtil);
+          storeCheckpointDir, storageConfig, dirDiffUtil) || getDeletedBlobs;

Review Comment:
   Please let me know if this sounds okay after our discussion. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] prateekm commented on a diff in pull request #1676: GetDeleted API and Recover from DeletedException

Posted by "prateekm (via GitHub)" <gi...@apache.org>.
prateekm commented on code in PR #1676:
URL: https://github.com/apache/samza/pull/1676#discussion_r1274120909


##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java:
##########
@@ -400,6 +349,7 @@ private void restoreStores() throws InterruptedException {
     });
 
     LOG.info("Store Restore complete");
+    return taskCheckpoints;

Review Comment:
   Don't you need to return the new checkpoint here?



##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManagerUtil.java:
##########
@@ -411,4 +438,292 @@ public static Set<String> getSideInputStoreNames(
     }
     return sideInputStores;
   }
+
+  public static List<Future<Void>> initAndRestoreTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, SamzaContainerMetrics samzaContainerMetrics,
+      CheckpointManager checkpointManager, JobContext jobContext, ContainerModel containerModel,
+      Map<TaskName, Checkpoint> taskCheckpoints, Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames,
+      Config config, ExecutorService executor, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      File loggerStoreDir, Map<String, SystemConsumer> storeConsumers) {
+
+    Set<String> forceRestoreTasks = new HashSet<>();
+    // Initialize each TaskStorageManager.
+    taskRestoreManagers.forEach((taskName, restoreManagers) ->
+        restoreManagers.forEach((factoryName, taskRestoreManager) -> {
+          try {
+            taskRestoreManager.init(taskCheckpoints.get(taskName));
+          } catch (SamzaException ex) {
+            if (isUnwrappedExceptionDeletedException(ex) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+              // Get deleted SnapshotIndex blob with GetDeleted and mark the task to be restored with GetDeleted as well.
+              // this ensures that the restore downloads the snapshot, recreates a new snapshot, uploads it to blob store
+              // and creates a new checkpoint.
+              ((BlobStoreRestoreManager) taskRestoreManager).init(taskCheckpoints.get(taskName), true);
+              forceRestoreTasks.add(taskName.getTaskName());
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("init failed for task: %s with GetDeleted set to true", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex);
+            }
+          }
+        })
+    );
+
+    // Start each store consumer once.
+    // Note: These consumers are per system and only changelog system store consumers will be started.
+    // Some TaskRestoreManagers may not require the consumer to be started, but due to the agnostic nature of
+    // ContainerStorageManager we always start the changelog consumer here in case it is required
+    storeConsumers.values().stream().distinct().forEach(SystemConsumer::start);
+
+    return restoreAllTaskInstances(taskRestoreManagers, taskCheckpoints, taskBackendFactoryToStoreNames, jobContext,
+        containerModel, samzaContainerMetrics, checkpointManager, config, taskInstanceMetrics, executor, loggerStoreDir,
+        forceRestoreTasks);
+  }
+
+  /**
+   * Restores all TaskInstances and returns a future for each TaskInstance restore. If this restore fails with DeletedException
+   * it will retry the restore with getDeleted flag, get all the blobs, recreate a new checkpoint in blob store and
+   * write that checkpoint to checkpoint topic.
+   * @param taskRestoreManagers map of TaskName to factory name to TaskRestoreManager map.
+   */
+  public static List<Future<Void>> restoreAllTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, Map<TaskName, Checkpoint> taskCheckpoints,
+      Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames, JobContext jobContext,
+      ContainerModel containerModel, SamzaContainerMetrics samzaContainerMetrics, CheckpointManager checkpointManager,
+      Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics, ExecutorService executor,
+      File loggedStoreDir, Set<String> forceRestoreTask) {
+
+    List<Future<Void>> taskRestoreFutures = new ArrayList<>();
+
+    // Submit restore callable for each taskInstance
+    taskRestoreManagers.forEach((taskInstanceName, restoreManagersMap) -> {
+      // Submit for each restore factory
+      restoreManagersMap.forEach((factoryName, taskRestoreManager) -> {
+        long startTime = System.currentTimeMillis();
+        String taskName = taskInstanceName.getTaskName();
+        LOG.info("Starting restore for state for task: {}", taskName);
+
+        CompletableFuture<Void> restoreFuture;
+        if (forceRestoreTask.contains(taskName) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+          restoreFuture = ((BlobStoreRestoreManager) taskRestoreManager).restore(true);
+        } else {
+          restoreFuture = taskRestoreManager.restore();
+        }
+
+        CompletableFuture<Void> taskRestoreFuture = restoreFuture.handle((res, ex) -> {
+          updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+
+          if (ex != null) {
+            if (isUnwrappedExceptionDeletedException(ex)) {
+              LOG.warn(
+                  "Restore state for task: {} received DeletedException. Reattempting with getDeletedBlobs enabled",

Review Comment:
   Minor: s/Restore state recieved XException/Got XException duringr estore for task/



##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManagerUtil.java:
##########
@@ -411,4 +438,292 @@ public static Set<String> getSideInputStoreNames(
     }
     return sideInputStores;
   }
+
+  public static List<Future<Void>> initAndRestoreTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, SamzaContainerMetrics samzaContainerMetrics,
+      CheckpointManager checkpointManager, JobContext jobContext, ContainerModel containerModel,
+      Map<TaskName, Checkpoint> taskCheckpoints, Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames,
+      Config config, ExecutorService executor, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      File loggerStoreDir, Map<String, SystemConsumer> storeConsumers) {
+
+    Set<String> forceRestoreTasks = new HashSet<>();
+    // Initialize each TaskStorageManager.
+    taskRestoreManagers.forEach((taskName, restoreManagers) ->
+        restoreManagers.forEach((factoryName, taskRestoreManager) -> {
+          try {
+            taskRestoreManager.init(taskCheckpoints.get(taskName));
+          } catch (SamzaException ex) {
+            if (isUnwrappedExceptionDeletedException(ex) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+              // Get deleted SnapshotIndex blob with GetDeleted and mark the task to be restored with GetDeleted as well.
+              // this ensures that the restore downloads the snapshot, recreates a new snapshot, uploads it to blob store
+              // and creates a new checkpoint.
+              ((BlobStoreRestoreManager) taskRestoreManager).init(taskCheckpoints.get(taskName), true);
+              forceRestoreTasks.add(taskName.getTaskName());
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("init failed for task: %s with GetDeleted set to true", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex);
+            }
+          }
+        })
+    );
+
+    // Start each store consumer once.
+    // Note: These consumers are per system and only changelog system store consumers will be started.
+    // Some TaskRestoreManagers may not require the consumer to be started, but due to the agnostic nature of
+    // ContainerStorageManager we always start the changelog consumer here in case it is required
+    storeConsumers.values().stream().distinct().forEach(SystemConsumer::start);
+
+    return restoreAllTaskInstances(taskRestoreManagers, taskCheckpoints, taskBackendFactoryToStoreNames, jobContext,
+        containerModel, samzaContainerMetrics, checkpointManager, config, taskInstanceMetrics, executor, loggerStoreDir,
+        forceRestoreTasks);
+  }
+
+  /**
+   * Restores all TaskInstances and returns a future for each TaskInstance restore. If this restore fails with DeletedException
+   * it will retry the restore with getDeleted flag, get all the blobs, recreate a new checkpoint in blob store and
+   * write that checkpoint to checkpoint topic.
+   * @param taskRestoreManagers map of TaskName to factory name to TaskRestoreManager map.
+   */
+  public static List<Future<Void>> restoreAllTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, Map<TaskName, Checkpoint> taskCheckpoints,
+      Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames, JobContext jobContext,
+      ContainerModel containerModel, SamzaContainerMetrics samzaContainerMetrics, CheckpointManager checkpointManager,
+      Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics, ExecutorService executor,
+      File loggedStoreDir, Set<String> forceRestoreTask) {
+
+    List<Future<Void>> taskRestoreFutures = new ArrayList<>();
+
+    // Submit restore callable for each taskInstance
+    taskRestoreManagers.forEach((taskInstanceName, restoreManagersMap) -> {
+      // Submit for each restore factory
+      restoreManagersMap.forEach((factoryName, taskRestoreManager) -> {
+        long startTime = System.currentTimeMillis();
+        String taskName = taskInstanceName.getTaskName();
+        LOG.info("Starting restore for state for task: {}", taskName);
+
+        CompletableFuture<Void> restoreFuture;
+        if (forceRestoreTask.contains(taskName) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+          restoreFuture = ((BlobStoreRestoreManager) taskRestoreManager).restore(true);
+        } else {
+          restoreFuture = taskRestoreManager.restore();
+        }
+
+        CompletableFuture<Void> taskRestoreFuture = restoreFuture.handle((res, ex) -> {
+          updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+
+          if (ex != null) {
+            if (isUnwrappedExceptionDeletedException(ex)) {
+              LOG.warn(
+                  "Restore state for task: {} received DeletedException. Reattempting with getDeletedBlobs enabled",
+                  taskInstanceName.getTaskName());
+
+              // Try to restore with getDeleted flag
+              CompletableFuture<Void> future =
+                  restoreTaskStoresAndCreateCheckpointWithGetDeleted(taskInstanceName, taskCheckpoints, checkpointManager,

Review Comment:
   Can these be two separate methods?
   restoreWithGetDeleted (returns checkpoint) and update/writeCheckpoint?



##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManagerUtil.java:
##########
@@ -411,4 +438,292 @@ public static Set<String> getSideInputStoreNames(
     }
     return sideInputStores;
   }
+
+  public static List<Future<Void>> initAndRestoreTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, SamzaContainerMetrics samzaContainerMetrics,
+      CheckpointManager checkpointManager, JobContext jobContext, ContainerModel containerModel,
+      Map<TaskName, Checkpoint> taskCheckpoints, Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames,
+      Config config, ExecutorService executor, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      File loggerStoreDir, Map<String, SystemConsumer> storeConsumers) {
+
+    Set<String> forceRestoreTasks = new HashSet<>();
+    // Initialize each TaskStorageManager.
+    taskRestoreManagers.forEach((taskName, restoreManagers) ->
+        restoreManagers.forEach((factoryName, taskRestoreManager) -> {
+          try {
+            taskRestoreManager.init(taskCheckpoints.get(taskName));
+          } catch (SamzaException ex) {
+            if (isUnwrappedExceptionDeletedException(ex) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+              // Get deleted SnapshotIndex blob with GetDeleted and mark the task to be restored with GetDeleted as well.
+              // this ensures that the restore downloads the snapshot, recreates a new snapshot, uploads it to blob store
+              // and creates a new checkpoint.
+              ((BlobStoreRestoreManager) taskRestoreManager).init(taskCheckpoints.get(taskName), true);
+              forceRestoreTasks.add(taskName.getTaskName());
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("init failed for task: %s with GetDeleted set to true", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex);
+            }
+          }
+        })
+    );
+
+    // Start each store consumer once.
+    // Note: These consumers are per system and only changelog system store consumers will be started.
+    // Some TaskRestoreManagers may not require the consumer to be started, but due to the agnostic nature of
+    // ContainerStorageManager we always start the changelog consumer here in case it is required
+    storeConsumers.values().stream().distinct().forEach(SystemConsumer::start);
+
+    return restoreAllTaskInstances(taskRestoreManagers, taskCheckpoints, taskBackendFactoryToStoreNames, jobContext,
+        containerModel, samzaContainerMetrics, checkpointManager, config, taskInstanceMetrics, executor, loggerStoreDir,
+        forceRestoreTasks);
+  }
+
+  /**
+   * Restores all TaskInstances and returns a future for each TaskInstance restore. If this restore fails with DeletedException
+   * it will retry the restore with getDeleted flag, get all the blobs, recreate a new checkpoint in blob store and
+   * write that checkpoint to checkpoint topic.
+   * @param taskRestoreManagers map of TaskName to factory name to TaskRestoreManager map.
+   */
+  public static List<Future<Void>> restoreAllTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, Map<TaskName, Checkpoint> taskCheckpoints,
+      Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames, JobContext jobContext,
+      ContainerModel containerModel, SamzaContainerMetrics samzaContainerMetrics, CheckpointManager checkpointManager,
+      Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics, ExecutorService executor,
+      File loggedStoreDir, Set<String> forceRestoreTask) {
+
+    List<Future<Void>> taskRestoreFutures = new ArrayList<>();
+
+    // Submit restore callable for each taskInstance
+    taskRestoreManagers.forEach((taskInstanceName, restoreManagersMap) -> {
+      // Submit for each restore factory
+      restoreManagersMap.forEach((factoryName, taskRestoreManager) -> {
+        long startTime = System.currentTimeMillis();
+        String taskName = taskInstanceName.getTaskName();
+        LOG.info("Starting restore for state for task: {}", taskName);
+
+        CompletableFuture<Void> restoreFuture;
+        if (forceRestoreTask.contains(taskName) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+          restoreFuture = ((BlobStoreRestoreManager) taskRestoreManager).restore(true);
+        } else {
+          restoreFuture = taskRestoreManager.restore();
+        }
+
+        CompletableFuture<Void> taskRestoreFuture = restoreFuture.handle((res, ex) -> {
+          updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+
+          if (ex != null) {
+            if (isUnwrappedExceptionDeletedException(ex)) {
+              LOG.warn(
+                  "Restore state for task: {} received DeletedException. Reattempting with getDeletedBlobs enabled",
+                  taskInstanceName.getTaskName());
+
+              // Try to restore with getDeleted flag
+              CompletableFuture<Void> future =
+                  restoreTaskStoresAndCreateCheckpointWithGetDeleted(taskInstanceName, taskCheckpoints, checkpointManager,
+                      taskRestoreManager, config, taskInstanceMetrics, executor,
+                      taskBackendFactoryToStoreNames.get(taskInstanceName).get(factoryName), loggedStoreDir, jobContext,
+                      containerModel);
+              try {
+                if (future != null) {
+                  future.join(); // Block and restore deleted state before continuing

Review Comment:
   Make sure this does not cause deadlocks. I.e. that the restore etc. are happening on the restore executor.



##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManagerUtil.java:
##########
@@ -411,4 +438,292 @@ public static Set<String> getSideInputStoreNames(
     }
     return sideInputStores;
   }
+
+  public static List<Future<Void>> initAndRestoreTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, SamzaContainerMetrics samzaContainerMetrics,
+      CheckpointManager checkpointManager, JobContext jobContext, ContainerModel containerModel,
+      Map<TaskName, Checkpoint> taskCheckpoints, Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames,
+      Config config, ExecutorService executor, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      File loggerStoreDir, Map<String, SystemConsumer> storeConsumers) {
+
+    Set<String> forceRestoreTasks = new HashSet<>();
+    // Initialize each TaskStorageManager.
+    taskRestoreManagers.forEach((taskName, restoreManagers) ->
+        restoreManagers.forEach((factoryName, taskRestoreManager) -> {
+          try {
+            taskRestoreManager.init(taskCheckpoints.get(taskName));
+          } catch (SamzaException ex) {
+            if (isUnwrappedExceptionDeletedException(ex) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+              // Get deleted SnapshotIndex blob with GetDeleted and mark the task to be restored with GetDeleted as well.
+              // this ensures that the restore downloads the snapshot, recreates a new snapshot, uploads it to blob store
+              // and creates a new checkpoint.
+              ((BlobStoreRestoreManager) taskRestoreManager).init(taskCheckpoints.get(taskName), true);
+              forceRestoreTasks.add(taskName.getTaskName());
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("init failed for task: %s with GetDeleted set to true", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex);
+            }
+          }
+        })
+    );
+
+    // Start each store consumer once.
+    // Note: These consumers are per system and only changelog system store consumers will be started.
+    // Some TaskRestoreManagers may not require the consumer to be started, but due to the agnostic nature of
+    // ContainerStorageManager we always start the changelog consumer here in case it is required
+    storeConsumers.values().stream().distinct().forEach(SystemConsumer::start);
+
+    return restoreAllTaskInstances(taskRestoreManagers, taskCheckpoints, taskBackendFactoryToStoreNames, jobContext,
+        containerModel, samzaContainerMetrics, checkpointManager, config, taskInstanceMetrics, executor, loggerStoreDir,
+        forceRestoreTasks);
+  }
+
+  /**
+   * Restores all TaskInstances and returns a future for each TaskInstance restore. If this restore fails with DeletedException
+   * it will retry the restore with getDeleted flag, get all the blobs, recreate a new checkpoint in blob store and
+   * write that checkpoint to checkpoint topic.
+   * @param taskRestoreManagers map of TaskName to factory name to TaskRestoreManager map.
+   */
+  public static List<Future<Void>> restoreAllTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, Map<TaskName, Checkpoint> taskCheckpoints,
+      Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames, JobContext jobContext,
+      ContainerModel containerModel, SamzaContainerMetrics samzaContainerMetrics, CheckpointManager checkpointManager,
+      Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics, ExecutorService executor,
+      File loggedStoreDir, Set<String> forceRestoreTask) {
+
+    List<Future<Void>> taskRestoreFutures = new ArrayList<>();
+
+    // Submit restore callable for each taskInstance
+    taskRestoreManagers.forEach((taskInstanceName, restoreManagersMap) -> {
+      // Submit for each restore factory
+      restoreManagersMap.forEach((factoryName, taskRestoreManager) -> {
+        long startTime = System.currentTimeMillis();
+        String taskName = taskInstanceName.getTaskName();
+        LOG.info("Starting restore for state for task: {}", taskName);
+
+        CompletableFuture<Void> restoreFuture;
+        if (forceRestoreTask.contains(taskName) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+          restoreFuture = ((BlobStoreRestoreManager) taskRestoreManager).restore(true);
+        } else {
+          restoreFuture = taskRestoreManager.restore();
+        }
+
+        CompletableFuture<Void> taskRestoreFuture = restoreFuture.handle((res, ex) -> {
+          updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+
+          if (ex != null) {
+            if (isUnwrappedExceptionDeletedException(ex)) {
+              LOG.warn(
+                  "Restore state for task: {} received DeletedException. Reattempting with getDeletedBlobs enabled",
+                  taskInstanceName.getTaskName());
+
+              // Try to restore with getDeleted flag
+              CompletableFuture<Void> future =
+                  restoreTaskStoresAndCreateCheckpointWithGetDeleted(taskInstanceName, taskCheckpoints, checkpointManager,
+                      taskRestoreManager, config, taskInstanceMetrics, executor,
+                      taskBackendFactoryToStoreNames.get(taskInstanceName).get(factoryName), loggedStoreDir, jobContext,
+                      containerModel);
+              try {
+                if (future != null) {
+                  future.join(); // Block and restore deleted state before continuing
+                }
+              } catch (Exception e) {
+                String msg = String.format("Unable to recover from DeletedException for task %s.", taskName);
+                throw new SamzaException(msg, e);
+              } finally {
+                updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+              }
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("Error restoring state for task: %s", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex); // wrap in unchecked exception to throw from lambda
+            }
+          }
+
+          // Stop all persistent stores after restoring. Certain persistent stores opened in BulkLoad mode are compacted
+          // on stop, so paralleling stop() also parallelizes their compaction (a time-intensive operation).
+          try {
+            taskRestoreManager.close();
+          } catch (Exception e) {
+            LOG.error("Error closing restore manager for task: {} after {} restore", taskName,
+                ex != null ? "unsuccessful" : "successful", e);
+            // ignore exception from close. container may still be able to continue processing/backups
+            // if restore manager close fails.
+          }
+          return null;
+        });
+        taskRestoreFutures.add(taskRestoreFuture);
+      });
+    });
+
+    return taskRestoreFutures;
+  }
+
+  public static CompletableFuture<Void> restoreTaskStoresAndCreateCheckpointWithGetDeleted(TaskName taskName,
+      Map<TaskName, Checkpoint> taskCheckpoints, CheckpointManager checkpointManager,
+      TaskRestoreManager taskRestoreManager, Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      ExecutorService executor, Set<String> storesToRestore, File loggedStoreBaseDirectory, JobContext jobContext,
+      ContainerModel containerModel) {
+
+    // if taskInstanceMetrics are specified use those for store metrics,
+    // otherwise (in case of StorageRecovery) use a blank MetricsRegistryMap
+    MetricsRegistry metricsRegistry =
+        taskInstanceMetrics.get(taskName) != null ? taskInstanceMetrics.get(taskName).registry()
+            : new MetricsRegistryMap();
+
+    BlobStoreManager blobStoreManager = getBlobStoreManager(config, executor);
+    JobConfig jobConfig = new JobConfig(config);
+    BlobStoreUtil blobStoreUtil =
+        new BlobStoreUtil(blobStoreManager, executor, new BlobStoreConfig(config), null,
+            new BlobStoreRestoreManagerMetrics(metricsRegistry));
+
+    BlobStoreRestoreManager blobStoreRestoreManager = (BlobStoreRestoreManager) taskRestoreManager;
+
+    CheckpointId checkpointId = CheckpointId.create();
+    Map<String, String> oldSCMs = ((CheckpointV2) taskCheckpoints.get(taskName))
+        .getStateCheckpointMarkers().get(BlobStoreStateBackendFactory.class.getName());
+
+    // Returns a single future that guarantees all the following are completed, in this order:
+    // 1. Restore state locally by getting deleted blobs from the blob store.
+    // 2. Create a new Checkpoint from restored state and back it up on the blob store.
+    // 3. Clean up old/deleted Snapshot
+    // 4. Remove TTL from the new Snapshot on the blob store
+    // 5. Write the new checkpoint to checkpoint topic and update taskCheckpoints map

Review Comment:
   Is this mutating the map passed in all the way from CSM? If so, strongly prefer returning new values instead of mutating params / incoming data structures.



##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManagerUtil.java:
##########
@@ -411,4 +438,292 @@ public static Set<String> getSideInputStoreNames(
     }
     return sideInputStores;
   }
+
+  public static List<Future<Void>> initAndRestoreTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, SamzaContainerMetrics samzaContainerMetrics,
+      CheckpointManager checkpointManager, JobContext jobContext, ContainerModel containerModel,
+      Map<TaskName, Checkpoint> taskCheckpoints, Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames,
+      Config config, ExecutorService executor, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      File loggerStoreDir, Map<String, SystemConsumer> storeConsumers) {
+
+    Set<String> forceRestoreTasks = new HashSet<>();
+    // Initialize each TaskStorageManager.
+    taskRestoreManagers.forEach((taskName, restoreManagers) ->
+        restoreManagers.forEach((factoryName, taskRestoreManager) -> {
+          try {
+            taskRestoreManager.init(taskCheckpoints.get(taskName));
+          } catch (SamzaException ex) {
+            if (isUnwrappedExceptionDeletedException(ex) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+              // Get deleted SnapshotIndex blob with GetDeleted and mark the task to be restored with GetDeleted as well.
+              // this ensures that the restore downloads the snapshot, recreates a new snapshot, uploads it to blob store
+              // and creates a new checkpoint.
+              ((BlobStoreRestoreManager) taskRestoreManager).init(taskCheckpoints.get(taskName), true);
+              forceRestoreTasks.add(taskName.getTaskName());
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("init failed for task: %s with GetDeleted set to true", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex);
+            }
+          }
+        })
+    );
+
+    // Start each store consumer once.
+    // Note: These consumers are per system and only changelog system store consumers will be started.
+    // Some TaskRestoreManagers may not require the consumer to be started, but due to the agnostic nature of
+    // ContainerStorageManager we always start the changelog consumer here in case it is required
+    storeConsumers.values().stream().distinct().forEach(SystemConsumer::start);
+
+    return restoreAllTaskInstances(taskRestoreManagers, taskCheckpoints, taskBackendFactoryToStoreNames, jobContext,
+        containerModel, samzaContainerMetrics, checkpointManager, config, taskInstanceMetrics, executor, loggerStoreDir,
+        forceRestoreTasks);
+  }
+
+  /**
+   * Restores all TaskInstances and returns a future for each TaskInstance restore. If this restore fails with DeletedException
+   * it will retry the restore with getDeleted flag, get all the blobs, recreate a new checkpoint in blob store and
+   * write that checkpoint to checkpoint topic.
+   * @param taskRestoreManagers map of TaskName to factory name to TaskRestoreManager map.
+   */
+  public static List<Future<Void>> restoreAllTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, Map<TaskName, Checkpoint> taskCheckpoints,
+      Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames, JobContext jobContext,
+      ContainerModel containerModel, SamzaContainerMetrics samzaContainerMetrics, CheckpointManager checkpointManager,
+      Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics, ExecutorService executor,
+      File loggedStoreDir, Set<String> forceRestoreTask) {
+
+    List<Future<Void>> taskRestoreFutures = new ArrayList<>();
+
+    // Submit restore callable for each taskInstance
+    taskRestoreManagers.forEach((taskInstanceName, restoreManagersMap) -> {
+      // Submit for each restore factory
+      restoreManagersMap.forEach((factoryName, taskRestoreManager) -> {
+        long startTime = System.currentTimeMillis();
+        String taskName = taskInstanceName.getTaskName();
+        LOG.info("Starting restore for state for task: {}", taskName);
+
+        CompletableFuture<Void> restoreFuture;
+        if (forceRestoreTask.contains(taskName) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+          restoreFuture = ((BlobStoreRestoreManager) taskRestoreManager).restore(true);
+        } else {
+          restoreFuture = taskRestoreManager.restore();
+        }
+
+        CompletableFuture<Void> taskRestoreFuture = restoreFuture.handle((res, ex) -> {
+          updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+
+          if (ex != null) {
+            if (isUnwrappedExceptionDeletedException(ex)) {
+              LOG.warn(
+                  "Restore state for task: {} received DeletedException. Reattempting with getDeletedBlobs enabled",
+                  taskInstanceName.getTaskName());
+
+              // Try to restore with getDeleted flag
+              CompletableFuture<Void> future =
+                  restoreTaskStoresAndCreateCheckpointWithGetDeleted(taskInstanceName, taskCheckpoints, checkpointManager,
+                      taskRestoreManager, config, taskInstanceMetrics, executor,
+                      taskBackendFactoryToStoreNames.get(taskInstanceName).get(factoryName), loggedStoreDir, jobContext,
+                      containerModel);
+              try {
+                if (future != null) {
+                  future.join(); // Block and restore deleted state before continuing
+                }
+              } catch (Exception e) {
+                String msg = String.format("Unable to recover from DeletedException for task %s.", taskName);
+                throw new SamzaException(msg, e);
+              } finally {
+                updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+              }
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("Error restoring state for task: %s", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex); // wrap in unchecked exception to throw from lambda
+            }
+          }
+
+          // Stop all persistent stores after restoring. Certain persistent stores opened in BulkLoad mode are compacted
+          // on stop, so paralleling stop() also parallelizes their compaction (a time-intensive operation).
+          try {
+            taskRestoreManager.close();
+          } catch (Exception e) {
+            LOG.error("Error closing restore manager for task: {} after {} restore", taskName,
+                ex != null ? "unsuccessful" : "successful", e);
+            // ignore exception from close. container may still be able to continue processing/backups
+            // if restore manager close fails.
+          }
+          return null;
+        });
+        taskRestoreFutures.add(taskRestoreFuture);
+      });
+    });
+
+    return taskRestoreFutures;
+  }
+
+  public static CompletableFuture<Void> restoreTaskStoresAndCreateCheckpointWithGetDeleted(TaskName taskName,
+      Map<TaskName, Checkpoint> taskCheckpoints, CheckpointManager checkpointManager,
+      TaskRestoreManager taskRestoreManager, Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      ExecutorService executor, Set<String> storesToRestore, File loggedStoreBaseDirectory, JobContext jobContext,
+      ContainerModel containerModel) {
+
+    // if taskInstanceMetrics are specified use those for store metrics,
+    // otherwise (in case of StorageRecovery) use a blank MetricsRegistryMap
+    MetricsRegistry metricsRegistry =
+        taskInstanceMetrics.get(taskName) != null ? taskInstanceMetrics.get(taskName).registry()
+            : new MetricsRegistryMap();
+
+    BlobStoreManager blobStoreManager = getBlobStoreManager(config, executor);
+    JobConfig jobConfig = new JobConfig(config);
+    BlobStoreUtil blobStoreUtil =
+        new BlobStoreUtil(blobStoreManager, executor, new BlobStoreConfig(config), null,
+            new BlobStoreRestoreManagerMetrics(metricsRegistry));
+
+    BlobStoreRestoreManager blobStoreRestoreManager = (BlobStoreRestoreManager) taskRestoreManager;
+
+    CheckpointId checkpointId = CheckpointId.create();
+    Map<String, String> oldSCMs = ((CheckpointV2) taskCheckpoints.get(taskName))
+        .getStateCheckpointMarkers().get(BlobStoreStateBackendFactory.class.getName());
+
+    // Returns a single future that guarantees all the following are completed, in this order:
+    // 1. Restore state locally by getting deleted blobs from the blob store.
+    // 2. Create a new Checkpoint from restored state and back it up on the blob store.
+    // 3. Clean up old/deleted Snapshot
+    // 4. Remove TTL from the new Snapshot on the blob store
+    // 5. Write the new checkpoint to checkpoint topic and update taskCheckpoints map
+
+    // 1. restore state with getDeleted flag set to true
+    return blobStoreRestoreManager.restore(true).thenCompose(r -> {
+      // 2. Create and backup new checkpoint on the blob store.
+      CompletableFuture<Map<String, String>> uploadSCMs =
+          createNewCheckpointAndBackupStores(jobContext, containerModel, config, taskName, storesToRestore, checkpointId,
+              loggedStoreBaseDirectory, blobStoreManager, metricsRegistry, executor);
+      CompletableFuture<Void> future = uploadSCMs.thenCompose(scms -> {
+        // 3. Delete prev SnapshotIndex including files/subdirs,
+        Map<String, CompletionStage<Void>> deleteOldSnapshotsFutures =
+            deletePrevSnapshots(blobStoreUtil, oldSCMs, taskName, jobConfig);
+        // 4. Mark new Snapshots to never expire
+        Map<String, CompletionStage<Void>> removeNewSnapshotsTTLFutures =
+            removeTTLNewSnapshots(blobStoreUtil, scms, taskName, jobConfig);
+        // Combined future of delete old snapshots and removeTTL of new snapshots
+        CompletableFuture<Void> removeTTLFuture = CompletableFuture.allOf(FutureUtil.mapToFuture(deleteOldSnapshotsFutures),
+            FutureUtil.mapToFuture(removeNewSnapshotsTTLFutures));
+        // 5. Update taskCheckpoints map with new checkpoint
+        CompletableFuture<Void> checkpointFuture = removeTTLFuture.thenRun(() -> {

Review Comment:
   should this be andThen instead of thenRun? thenRun is fire and forget and is not guaranteed to complete when this future completes.
   
   Generally, should not thenCompose recursively in error handlers, e.g. line 602. Chain on the original future to clarify execution DAG.
   
   Also, do you need to specify the executor these operations are happening on?



##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManagerUtil.java:
##########
@@ -411,4 +438,292 @@ public static Set<String> getSideInputStoreNames(
     }
     return sideInputStores;
   }
+
+  public static List<Future<Void>> initAndRestoreTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, SamzaContainerMetrics samzaContainerMetrics,
+      CheckpointManager checkpointManager, JobContext jobContext, ContainerModel containerModel,
+      Map<TaskName, Checkpoint> taskCheckpoints, Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames,
+      Config config, ExecutorService executor, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      File loggerStoreDir, Map<String, SystemConsumer> storeConsumers) {
+
+    Set<String> forceRestoreTasks = new HashSet<>();
+    // Initialize each TaskStorageManager.
+    taskRestoreManagers.forEach((taskName, restoreManagers) ->
+        restoreManagers.forEach((factoryName, taskRestoreManager) -> {
+          try {
+            taskRestoreManager.init(taskCheckpoints.get(taskName));
+          } catch (SamzaException ex) {
+            if (isUnwrappedExceptionDeletedException(ex) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+              // Get deleted SnapshotIndex blob with GetDeleted and mark the task to be restored with GetDeleted as well.
+              // this ensures that the restore downloads the snapshot, recreates a new snapshot, uploads it to blob store
+              // and creates a new checkpoint.
+              ((BlobStoreRestoreManager) taskRestoreManager).init(taskCheckpoints.get(taskName), true);
+              forceRestoreTasks.add(taskName.getTaskName());
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("init failed for task: %s with GetDeleted set to true", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex);
+            }
+          }
+        })
+    );
+
+    // Start each store consumer once.
+    // Note: These consumers are per system and only changelog system store consumers will be started.
+    // Some TaskRestoreManagers may not require the consumer to be started, but due to the agnostic nature of
+    // ContainerStorageManager we always start the changelog consumer here in case it is required
+    storeConsumers.values().stream().distinct().forEach(SystemConsumer::start);
+
+    return restoreAllTaskInstances(taskRestoreManagers, taskCheckpoints, taskBackendFactoryToStoreNames, jobContext,
+        containerModel, samzaContainerMetrics, checkpointManager, config, taskInstanceMetrics, executor, loggerStoreDir,
+        forceRestoreTasks);
+  }
+
+  /**
+   * Restores all TaskInstances and returns a future for each TaskInstance restore. If this restore fails with DeletedException
+   * it will retry the restore with getDeleted flag, get all the blobs, recreate a new checkpoint in blob store and
+   * write that checkpoint to checkpoint topic.
+   * @param taskRestoreManagers map of TaskName to factory name to TaskRestoreManager map.
+   */
+  public static List<Future<Void>> restoreAllTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, Map<TaskName, Checkpoint> taskCheckpoints,
+      Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames, JobContext jobContext,
+      ContainerModel containerModel, SamzaContainerMetrics samzaContainerMetrics, CheckpointManager checkpointManager,
+      Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics, ExecutorService executor,
+      File loggedStoreDir, Set<String> forceRestoreTask) {
+
+    List<Future<Void>> taskRestoreFutures = new ArrayList<>();
+
+    // Submit restore callable for each taskInstance
+    taskRestoreManagers.forEach((taskInstanceName, restoreManagersMap) -> {
+      // Submit for each restore factory
+      restoreManagersMap.forEach((factoryName, taskRestoreManager) -> {
+        long startTime = System.currentTimeMillis();
+        String taskName = taskInstanceName.getTaskName();
+        LOG.info("Starting restore for state for task: {}", taskName);
+
+        CompletableFuture<Void> restoreFuture;
+        if (forceRestoreTask.contains(taskName) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+          restoreFuture = ((BlobStoreRestoreManager) taskRestoreManager).restore(true);
+        } else {
+          restoreFuture = taskRestoreManager.restore();
+        }
+
+        CompletableFuture<Void> taskRestoreFuture = restoreFuture.handle((res, ex) -> {
+          updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+
+          if (ex != null) {
+            if (isUnwrappedExceptionDeletedException(ex)) {
+              LOG.warn(
+                  "Restore state for task: {} received DeletedException. Reattempting with getDeletedBlobs enabled",
+                  taskInstanceName.getTaskName());
+
+              // Try to restore with getDeleted flag
+              CompletableFuture<Void> future =
+                  restoreTaskStoresAndCreateCheckpointWithGetDeleted(taskInstanceName, taskCheckpoints, checkpointManager,
+                      taskRestoreManager, config, taskInstanceMetrics, executor,
+                      taskBackendFactoryToStoreNames.get(taskInstanceName).get(factoryName), loggedStoreDir, jobContext,
+                      containerModel);
+              try {
+                if (future != null) {
+                  future.join(); // Block and restore deleted state before continuing
+                }
+              } catch (Exception e) {
+                String msg = String.format("Unable to recover from DeletedException for task %s.", taskName);
+                throw new SamzaException(msg, e);
+              } finally {
+                updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+              }
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("Error restoring state for task: %s", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex); // wrap in unchecked exception to throw from lambda
+            }
+          }
+
+          // Stop all persistent stores after restoring. Certain persistent stores opened in BulkLoad mode are compacted
+          // on stop, so paralleling stop() also parallelizes their compaction (a time-intensive operation).
+          try {
+            taskRestoreManager.close();
+          } catch (Exception e) {
+            LOG.error("Error closing restore manager for task: {} after {} restore", taskName,
+                ex != null ? "unsuccessful" : "successful", e);
+            // ignore exception from close. container may still be able to continue processing/backups
+            // if restore manager close fails.
+          }
+          return null;
+        });
+        taskRestoreFutures.add(taskRestoreFuture);
+      });
+    });
+
+    return taskRestoreFutures;
+  }
+
+  public static CompletableFuture<Void> restoreTaskStoresAndCreateCheckpointWithGetDeleted(TaskName taskName,
+      Map<TaskName, Checkpoint> taskCheckpoints, CheckpointManager checkpointManager,
+      TaskRestoreManager taskRestoreManager, Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      ExecutorService executor, Set<String> storesToRestore, File loggedStoreBaseDirectory, JobContext jobContext,
+      ContainerModel containerModel) {
+
+    // if taskInstanceMetrics are specified use those for store metrics,
+    // otherwise (in case of StorageRecovery) use a blank MetricsRegistryMap
+    MetricsRegistry metricsRegistry =
+        taskInstanceMetrics.get(taskName) != null ? taskInstanceMetrics.get(taskName).registry()
+            : new MetricsRegistryMap();
+
+    BlobStoreManager blobStoreManager = getBlobStoreManager(config, executor);
+    JobConfig jobConfig = new JobConfig(config);
+    BlobStoreUtil blobStoreUtil =
+        new BlobStoreUtil(blobStoreManager, executor, new BlobStoreConfig(config), null,
+            new BlobStoreRestoreManagerMetrics(metricsRegistry));
+
+    BlobStoreRestoreManager blobStoreRestoreManager = (BlobStoreRestoreManager) taskRestoreManager;
+
+    CheckpointId checkpointId = CheckpointId.create();
+    Map<String, String> oldSCMs = ((CheckpointV2) taskCheckpoints.get(taskName))
+        .getStateCheckpointMarkers().get(BlobStoreStateBackendFactory.class.getName());
+
+    // Returns a single future that guarantees all the following are completed, in this order:
+    // 1. Restore state locally by getting deleted blobs from the blob store.
+    // 2. Create a new Checkpoint from restored state and back it up on the blob store.
+    // 3. Clean up old/deleted Snapshot
+    // 4. Remove TTL from the new Snapshot on the blob store
+    // 5. Write the new checkpoint to checkpoint topic and update taskCheckpoints map
+
+    // 1. restore state with getDeleted flag set to true
+    return blobStoreRestoreManager.restore(true).thenCompose(r -> {
+      // 2. Create and backup new checkpoint on the blob store.
+      CompletableFuture<Map<String, String>> uploadSCMs =
+          createNewCheckpointAndBackupStores(jobContext, containerModel, config, taskName, storesToRestore, checkpointId,
+              loggedStoreBaseDirectory, blobStoreManager, metricsRegistry, executor);
+      CompletableFuture<Void> future = uploadSCMs.thenCompose(scms -> {
+        // 3. Delete prev SnapshotIndex including files/subdirs,
+        Map<String, CompletionStage<Void>> deleteOldSnapshotsFutures =
+            deletePrevSnapshots(blobStoreUtil, oldSCMs, taskName, jobConfig);
+        // 4. Mark new Snapshots to never expire
+        Map<String, CompletionStage<Void>> removeNewSnapshotsTTLFutures =
+            removeTTLNewSnapshots(blobStoreUtil, scms, taskName, jobConfig);
+        // Combined future of delete old snapshots and removeTTL of new snapshots
+        CompletableFuture<Void> removeTTLFuture = CompletableFuture.allOf(FutureUtil.mapToFuture(deleteOldSnapshotsFutures),
+            FutureUtil.mapToFuture(removeNewSnapshotsTTLFutures));
+        // 5. Update taskCheckpoints map with new checkpoint
+        CompletableFuture<Void> checkpointFuture = removeTTLFuture.thenRun(() -> {
+          Checkpoint newCheckpoint = writeNewCheckpoint(taskName, checkpointId, scms, checkpointManager);
+          taskCheckpoints.put(taskName, newCheckpoint);
+        });
+        return checkpointFuture;
+      });
+      return future;
+    }).exceptionally(ex -> {
+      String msg = String.format("Restore for task %s failed with get deleted bob", taskName);
+      throw new SamzaException(msg, ex);
+    });
+  }
+
+  private static CompletableFuture<Map<String, String>> createNewCheckpointAndBackupStores(JobContext jobContext,
+      ContainerModel containerModel, Config config, TaskName taskName, Set<String> storesToBackup,
+      CheckpointId newCheckpointId, File loggedStoreBaseDirectory, BlobStoreManager blobStoreManager,
+      MetricsRegistry metricsRegistry, ExecutorService executor) {
+
+    BlobStoreBackupManagerMetrics blobStoreBackupManagerMetrics = new BlobStoreBackupManagerMetrics(metricsRegistry);
+    BlobStoreBackupManager blobStoreBackupManager =
+        new BlobStoreBackupManager(jobContext.getJobModel(), containerModel, containerModel.getTasks().get(taskName),
+            executor, blobStoreBackupManagerMetrics, config, SystemClock.instance(), loggedStoreBaseDirectory,
+            new StorageManagerUtil(), blobStoreManager);
+
+    // create checkpoint dir as a copy of store dir
+    createCheckpointDirFromStoreDirCopy(taskName, containerModel.getTasks().get(taskName),
+        loggedStoreBaseDirectory, storesToBackup, newCheckpointId);
+    // upload to blob store and return future
+    return blobStoreBackupManager.upload(newCheckpointId, new HashMap<>());
+  }
+
+  private static Checkpoint writeNewCheckpoint(TaskName taskName, CheckpointId checkpointId,
+      Map<String, String> newSCMs, CheckpointManager checkpointManager) {
+    CheckpointV2 oldCheckpoint = (CheckpointV2) checkpointManager.readLastCheckpoint(taskName);
+    Map<SystemStreamPartition, String> checkpointOffsets = oldCheckpoint.getOffsets();
+
+    Map<String, Map<String, String>> oldStateCheckpointMarkers = oldCheckpoint.getStateCheckpointMarkers();
+
+    Map<String, Map<String, String>> newStateCheckpointMarkers = ImmutableMap.<String, Map<String, String>>builder()
+        .put(KafkaChangelogStateBackendFactory.class.getName(), oldStateCheckpointMarkers.get(KafkaChangelogStateBackendFactory.class.getName()))
+        .put(BlobStoreStateBackendFactory.class.getName(), newSCMs)
+        .build();
+
+    CheckpointV2 checkpointV2 = new CheckpointV2(checkpointId, checkpointOffsets, newStateCheckpointMarkers);
+    checkpointManager.writeCheckpoint(taskName, checkpointV2);
+    return checkpointV2;
+  }
+
+  private static void createCheckpointDirFromStoreDirCopy(TaskName taskName, TaskModel taskModel,
+      File loggedStoreBaseDir, Set<String> storeName, CheckpointId checkpointId) {
+    StorageManagerUtil storageManagerUtil = new StorageManagerUtil();
+    for (String store : storeName) {
+      try {
+        File storeDirectory =
+            storageManagerUtil.getTaskStoreDir(loggedStoreBaseDir, store, taskName, taskModel.getTaskMode());
+        File checkpointDir = new File(storageManagerUtil.getStoreCheckpointDir(storeDirectory, checkpointId));
+        FileUtils.copyDirectory(storeDirectory, checkpointDir);
+      } catch (IOException exception) {
+        String msg = String.format("Unable to create a copy of store directory %s into checkpoint dir %s while "
+            + "attempting to recover from DeletedException", store, checkpointId);
+        throw new SamzaException(msg, exception);
+      }
+    }
+  }
+
+  private static BlobStoreManager getBlobStoreManager(Config config, ExecutorService executor) {
+    BlobStoreConfig blobStoreConfig = new BlobStoreConfig(config);
+    String blobStoreManagerFactory = blobStoreConfig.getBlobStoreManagerFactory();
+    BlobStoreManagerFactory factory = ReflectionUtil.getObj(blobStoreManagerFactory, BlobStoreManagerFactory.class);
+    return factory.getRestoreBlobStoreManager(config, executor);
+  }
+
+  private static void updateRestoreTime(long startTime, SamzaContainerMetrics samzaContainerMetrics,
+      TaskName taskInstance) {
+    long timeToRestore = System.currentTimeMillis() - startTime;
+    if (samzaContainerMetrics != null) {
+      Gauge taskGauge = samzaContainerMetrics.taskStoreRestorationMetrics().getOrDefault(taskInstance, null);
+
+      if (taskGauge != null) {
+        taskGauge.set(timeToRestore);
+      }
+    }
+  }
+
+  private static Boolean isUnwrappedExceptionDeletedException(Throwable ex) {
+    Throwable unwrappedException = FutureUtil.unwrapExceptions(CompletionException.class,
+        FutureUtil.unwrapExceptions(SamzaException.class, ex));
+    return unwrappedException instanceof DeletedException;
+  }
+
+  private static Metadata createSnapshotMetadataRequest(TaskName taskName, JobConfig jobConfig, String store) {

Review Comment:
   Typo? What request is being created?



##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManagerUtil.java:
##########
@@ -411,4 +438,292 @@ public static Set<String> getSideInputStoreNames(
     }
     return sideInputStores;
   }
+
+  public static List<Future<Void>> initAndRestoreTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, SamzaContainerMetrics samzaContainerMetrics,
+      CheckpointManager checkpointManager, JobContext jobContext, ContainerModel containerModel,
+      Map<TaskName, Checkpoint> taskCheckpoints, Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames,
+      Config config, ExecutorService executor, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      File loggerStoreDir, Map<String, SystemConsumer> storeConsumers) {
+
+    Set<String> forceRestoreTasks = new HashSet<>();
+    // Initialize each TaskStorageManager.
+    taskRestoreManagers.forEach((taskName, restoreManagers) ->
+        restoreManagers.forEach((factoryName, taskRestoreManager) -> {
+          try {
+            taskRestoreManager.init(taskCheckpoints.get(taskName));
+          } catch (SamzaException ex) {
+            if (isUnwrappedExceptionDeletedException(ex) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+              // Get deleted SnapshotIndex blob with GetDeleted and mark the task to be restored with GetDeleted as well.
+              // this ensures that the restore downloads the snapshot, recreates a new snapshot, uploads it to blob store
+              // and creates a new checkpoint.
+              ((BlobStoreRestoreManager) taskRestoreManager).init(taskCheckpoints.get(taskName), true);
+              forceRestoreTasks.add(taskName.getTaskName());
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("init failed for task: %s with GetDeleted set to true", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex);
+            }
+          }
+        })
+    );
+
+    // Start each store consumer once.
+    // Note: These consumers are per system and only changelog system store consumers will be started.
+    // Some TaskRestoreManagers may not require the consumer to be started, but due to the agnostic nature of
+    // ContainerStorageManager we always start the changelog consumer here in case it is required
+    storeConsumers.values().stream().distinct().forEach(SystemConsumer::start);
+
+    return restoreAllTaskInstances(taskRestoreManagers, taskCheckpoints, taskBackendFactoryToStoreNames, jobContext,
+        containerModel, samzaContainerMetrics, checkpointManager, config, taskInstanceMetrics, executor, loggerStoreDir,
+        forceRestoreTasks);
+  }
+
+  /**
+   * Restores all TaskInstances and returns a future for each TaskInstance restore. If this restore fails with DeletedException
+   * it will retry the restore with getDeleted flag, get all the blobs, recreate a new checkpoint in blob store and
+   * write that checkpoint to checkpoint topic.
+   * @param taskRestoreManagers map of TaskName to factory name to TaskRestoreManager map.
+   */
+  public static List<Future<Void>> restoreAllTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, Map<TaskName, Checkpoint> taskCheckpoints,
+      Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames, JobContext jobContext,
+      ContainerModel containerModel, SamzaContainerMetrics samzaContainerMetrics, CheckpointManager checkpointManager,
+      Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics, ExecutorService executor,
+      File loggedStoreDir, Set<String> forceRestoreTask) {
+
+    List<Future<Void>> taskRestoreFutures = new ArrayList<>();
+
+    // Submit restore callable for each taskInstance
+    taskRestoreManagers.forEach((taskInstanceName, restoreManagersMap) -> {
+      // Submit for each restore factory
+      restoreManagersMap.forEach((factoryName, taskRestoreManager) -> {
+        long startTime = System.currentTimeMillis();
+        String taskName = taskInstanceName.getTaskName();
+        LOG.info("Starting restore for state for task: {}", taskName);
+
+        CompletableFuture<Void> restoreFuture;
+        if (forceRestoreTask.contains(taskName) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+          restoreFuture = ((BlobStoreRestoreManager) taskRestoreManager).restore(true);
+        } else {
+          restoreFuture = taskRestoreManager.restore();
+        }
+
+        CompletableFuture<Void> taskRestoreFuture = restoreFuture.handle((res, ex) -> {
+          updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+
+          if (ex != null) {
+            if (isUnwrappedExceptionDeletedException(ex)) {
+              LOG.warn(
+                  "Restore state for task: {} received DeletedException. Reattempting with getDeletedBlobs enabled",
+                  taskInstanceName.getTaskName());
+
+              // Try to restore with getDeleted flag
+              CompletableFuture<Void> future =
+                  restoreTaskStoresAndCreateCheckpointWithGetDeleted(taskInstanceName, taskCheckpoints, checkpointManager,
+                      taskRestoreManager, config, taskInstanceMetrics, executor,
+                      taskBackendFactoryToStoreNames.get(taskInstanceName).get(factoryName), loggedStoreDir, jobContext,
+                      containerModel);
+              try {
+                if (future != null) {
+                  future.join(); // Block and restore deleted state before continuing
+                }
+              } catch (Exception e) {
+                String msg = String.format("Unable to recover from DeletedException for task %s.", taskName);
+                throw new SamzaException(msg, e);
+              } finally {
+                updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+              }
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("Error restoring state for task: %s", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex); // wrap in unchecked exception to throw from lambda
+            }
+          }
+
+          // Stop all persistent stores after restoring. Certain persistent stores opened in BulkLoad mode are compacted
+          // on stop, so paralleling stop() also parallelizes their compaction (a time-intensive operation).
+          try {
+            taskRestoreManager.close();
+          } catch (Exception e) {
+            LOG.error("Error closing restore manager for task: {} after {} restore", taskName,
+                ex != null ? "unsuccessful" : "successful", e);
+            // ignore exception from close. container may still be able to continue processing/backups
+            // if restore manager close fails.
+          }
+          return null;
+        });
+        taskRestoreFutures.add(taskRestoreFuture);
+      });
+    });
+
+    return taskRestoreFutures;
+  }
+
+  public static CompletableFuture<Void> restoreTaskStoresAndCreateCheckpointWithGetDeleted(TaskName taskName,
+      Map<TaskName, Checkpoint> taskCheckpoints, CheckpointManager checkpointManager,
+      TaskRestoreManager taskRestoreManager, Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      ExecutorService executor, Set<String> storesToRestore, File loggedStoreBaseDirectory, JobContext jobContext,
+      ContainerModel containerModel) {
+
+    // if taskInstanceMetrics are specified use those for store metrics,
+    // otherwise (in case of StorageRecovery) use a blank MetricsRegistryMap
+    MetricsRegistry metricsRegistry =
+        taskInstanceMetrics.get(taskName) != null ? taskInstanceMetrics.get(taskName).registry()
+            : new MetricsRegistryMap();
+
+    BlobStoreManager blobStoreManager = getBlobStoreManager(config, executor);
+    JobConfig jobConfig = new JobConfig(config);
+    BlobStoreUtil blobStoreUtil =
+        new BlobStoreUtil(blobStoreManager, executor, new BlobStoreConfig(config), null,
+            new BlobStoreRestoreManagerMetrics(metricsRegistry));
+
+    BlobStoreRestoreManager blobStoreRestoreManager = (BlobStoreRestoreManager) taskRestoreManager;
+
+    CheckpointId checkpointId = CheckpointId.create();
+    Map<String, String> oldSCMs = ((CheckpointV2) taskCheckpoints.get(taskName))
+        .getStateCheckpointMarkers().get(BlobStoreStateBackendFactory.class.getName());
+
+    // Returns a single future that guarantees all the following are completed, in this order:
+    // 1. Restore state locally by getting deleted blobs from the blob store.
+    // 2. Create a new Checkpoint from restored state and back it up on the blob store.
+    // 3. Clean up old/deleted Snapshot
+    // 4. Remove TTL from the new Snapshot on the blob store
+    // 5. Write the new checkpoint to checkpoint topic and update taskCheckpoints map
+
+    // 1. restore state with getDeleted flag set to true
+    return blobStoreRestoreManager.restore(true).thenCompose(r -> {
+      // 2. Create and backup new checkpoint on the blob store.
+      CompletableFuture<Map<String, String>> uploadSCMs =
+          createNewCheckpointAndBackupStores(jobContext, containerModel, config, taskName, storesToRestore, checkpointId,
+              loggedStoreBaseDirectory, blobStoreManager, metricsRegistry, executor);
+      CompletableFuture<Void> future = uploadSCMs.thenCompose(scms -> {
+        // 3. Delete prev SnapshotIndex including files/subdirs,
+        Map<String, CompletionStage<Void>> deleteOldSnapshotsFutures =
+            deletePrevSnapshots(blobStoreUtil, oldSCMs, taskName, jobConfig);
+        // 4. Mark new Snapshots to never expire
+        Map<String, CompletionStage<Void>> removeNewSnapshotsTTLFutures =
+            removeTTLNewSnapshots(blobStoreUtil, scms, taskName, jobConfig);
+        // Combined future of delete old snapshots and removeTTL of new snapshots
+        CompletableFuture<Void> removeTTLFuture = CompletableFuture.allOf(FutureUtil.mapToFuture(deleteOldSnapshotsFutures),
+            FutureUtil.mapToFuture(removeNewSnapshotsTTLFutures));
+        // 5. Update taskCheckpoints map with new checkpoint
+        CompletableFuture<Void> checkpointFuture = removeTTLFuture.thenRun(() -> {
+          Checkpoint newCheckpoint = writeNewCheckpoint(taskName, checkpointId, scms, checkpointManager);
+          taskCheckpoints.put(taskName, newCheckpoint);
+        });
+        return checkpointFuture;
+      });
+      return future;
+    }).exceptionally(ex -> {
+      String msg = String.format("Restore for task %s failed with get deleted bob", taskName);

Review Comment:
   Let's not use "with get deleted blob" as a top level message since it's impl detail (applies to other similar messages too). Explain the situation instead, e.g. "Could not restore state for task %s despite trying to recover deleted blobs". or something like that.



##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java:
##########
@@ -301,64 +300,14 @@ private void restoreStores() throws InterruptedException {
               samzaContainerMetrics, taskInstanceMetrics, taskInstanceCollectors, serdes,
               loggedStoreBaseDirectory, nonLoggedStoreBaseDirectory, config, clock);
       taskRestoreManagers.put(taskName, taskStoreRestoreManagers);
+      taskBackendFactoryToStoreNames.put(taskName, backendFactoryToStoreNames);
     });
 
-    // Initialize each TaskStorageManager
-    taskRestoreManagers.forEach((taskName, restoreManagers) ->
-        restoreManagers.forEach((factoryName, taskRestoreManager) ->
-            taskRestoreManager.init(taskCheckpoints.get(taskName))
-        )
-    );
-
-    // Start each store consumer once.
-    // Note: These consumers are per system and only changelog system store consumers will be started.
-    // Some TaskRestoreManagers may not require the consumer to to be started, but due to the agnostic nature of
-    // ContainerStorageManager we always start the changelog consumer here in case it is required
-    this.storeConsumers.values().stream().distinct().forEach(SystemConsumer::start);
-
-    List<Future<Void>> taskRestoreFutures = new ArrayList<>();
-
-    // Submit restore callable for each taskInstance
-    taskRestoreManagers.forEach((taskInstance, restoreManagersMap) -> {
-      // Submit for each restore factory
-      restoreManagersMap.forEach((factoryName, taskRestoreManager) -> {
-        long startTime = System.currentTimeMillis();
-        String taskName = taskInstance.getTaskName();
-        LOG.info("Starting restore for state for task: {}", taskName);
-        CompletableFuture<Void> restoreFuture = taskRestoreManager.restore().handle((res, ex) -> {
-          // Stop all persistent stores after restoring. Certain persistent stores opened in BulkLoad mode are compacted
-          // on stop, so paralleling stop() also parallelizes their compaction (a time-intensive operation).
-          try {
-            taskRestoreManager.close();
-          } catch (Exception e) {
-            LOG.error("Error closing restore manager for task: {} after {} restore",
-                taskName, ex != null ? "unsuccessful" : "successful", e);
-            // ignore exception from close. container may still be be able to continue processing/backups
-            // if restore manager close fails.
-          }
-
-          long timeToRestore = System.currentTimeMillis() - startTime;
-          if (samzaContainerMetrics != null) {
-            Gauge taskGauge = samzaContainerMetrics.taskStoreRestorationMetrics().getOrDefault(taskInstance, null);
-
-            if (taskGauge != null) {
-              taskGauge.set(timeToRestore);
-            }
-          }
-
-          if (ex != null) {
-            // log and rethrow exception to communicate restore failure
-            String msg = String.format("Error restoring state for task: %s", taskName);
-            LOG.error(msg, ex);
-            throw new SamzaException(msg, ex); // wrap in unchecked exception to throw from lambda
-          } else {
-            return null;
-          }
-        });
-
-        taskRestoreFutures.add(restoreFuture);
-      });
-    });
+    // Init all taskRestores and if successful, create a future for restores for each task
+    List<Future<Void>> taskRestoreFutures =

Review Comment:
   Blocker: Don't you need to return a List<Future<Map<Task, Checkpoints>>>  here, that contains the updated checkpoint with reuploaded SCM?



##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManagerUtil.java:
##########
@@ -411,4 +438,292 @@ public static Set<String> getSideInputStoreNames(
     }
     return sideInputStores;
   }
+
+  public static List<Future<Void>> initAndRestoreTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, SamzaContainerMetrics samzaContainerMetrics,
+      CheckpointManager checkpointManager, JobContext jobContext, ContainerModel containerModel,
+      Map<TaskName, Checkpoint> taskCheckpoints, Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames,
+      Config config, ExecutorService executor, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      File loggerStoreDir, Map<String, SystemConsumer> storeConsumers) {
+
+    Set<String> forceRestoreTasks = new HashSet<>();
+    // Initialize each TaskStorageManager.
+    taskRestoreManagers.forEach((taskName, restoreManagers) ->
+        restoreManagers.forEach((factoryName, taskRestoreManager) -> {
+          try {
+            taskRestoreManager.init(taskCheckpoints.get(taskName));
+          } catch (SamzaException ex) {
+            if (isUnwrappedExceptionDeletedException(ex) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+              // Get deleted SnapshotIndex blob with GetDeleted and mark the task to be restored with GetDeleted as well.
+              // this ensures that the restore downloads the snapshot, recreates a new snapshot, uploads it to blob store
+              // and creates a new checkpoint.
+              ((BlobStoreRestoreManager) taskRestoreManager).init(taskCheckpoints.get(taskName), true);
+              forceRestoreTasks.add(taskName.getTaskName());
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("init failed for task: %s with GetDeleted set to true", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex);
+            }
+          }
+        })
+    );
+
+    // Start each store consumer once.
+    // Note: These consumers are per system and only changelog system store consumers will be started.
+    // Some TaskRestoreManagers may not require the consumer to be started, but due to the agnostic nature of
+    // ContainerStorageManager we always start the changelog consumer here in case it is required
+    storeConsumers.values().stream().distinct().forEach(SystemConsumer::start);
+
+    return restoreAllTaskInstances(taskRestoreManagers, taskCheckpoints, taskBackendFactoryToStoreNames, jobContext,
+        containerModel, samzaContainerMetrics, checkpointManager, config, taskInstanceMetrics, executor, loggerStoreDir,
+        forceRestoreTasks);
+  }
+
+  /**
+   * Restores all TaskInstances and returns a future for each TaskInstance restore. If this restore fails with DeletedException
+   * it will retry the restore with getDeleted flag, get all the blobs, recreate a new checkpoint in blob store and
+   * write that checkpoint to checkpoint topic.
+   * @param taskRestoreManagers map of TaskName to factory name to TaskRestoreManager map.
+   */
+  public static List<Future<Void>> restoreAllTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, Map<TaskName, Checkpoint> taskCheckpoints,
+      Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames, JobContext jobContext,
+      ContainerModel containerModel, SamzaContainerMetrics samzaContainerMetrics, CheckpointManager checkpointManager,
+      Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics, ExecutorService executor,
+      File loggedStoreDir, Set<String> forceRestoreTask) {
+
+    List<Future<Void>> taskRestoreFutures = new ArrayList<>();
+
+    // Submit restore callable for each taskInstance
+    taskRestoreManagers.forEach((taskInstanceName, restoreManagersMap) -> {
+      // Submit for each restore factory
+      restoreManagersMap.forEach((factoryName, taskRestoreManager) -> {
+        long startTime = System.currentTimeMillis();
+        String taskName = taskInstanceName.getTaskName();
+        LOG.info("Starting restore for state for task: {}", taskName);
+
+        CompletableFuture<Void> restoreFuture;
+        if (forceRestoreTask.contains(taskName) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+          restoreFuture = ((BlobStoreRestoreManager) taskRestoreManager).restore(true);
+        } else {
+          restoreFuture = taskRestoreManager.restore();
+        }
+
+        CompletableFuture<Void> taskRestoreFuture = restoreFuture.handle((res, ex) -> {
+          updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+
+          if (ex != null) {
+            if (isUnwrappedExceptionDeletedException(ex)) {
+              LOG.warn(
+                  "Restore state for task: {} received DeletedException. Reattempting with getDeletedBlobs enabled",
+                  taskInstanceName.getTaskName());
+
+              // Try to restore with getDeleted flag
+              CompletableFuture<Void> future =
+                  restoreTaskStoresAndCreateCheckpointWithGetDeleted(taskInstanceName, taskCheckpoints, checkpointManager,
+                      taskRestoreManager, config, taskInstanceMetrics, executor,
+                      taskBackendFactoryToStoreNames.get(taskInstanceName).get(factoryName), loggedStoreDir, jobContext,
+                      containerModel);
+              try {
+                if (future != null) {
+                  future.join(); // Block and restore deleted state before continuing
+                }
+              } catch (Exception e) {
+                String msg = String.format("Unable to recover from DeletedException for task %s.", taskName);

Review Comment:
   Minor: DeletedException during restore with getDeleted ...



##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManagerUtil.java:
##########
@@ -411,4 +438,292 @@ public static Set<String> getSideInputStoreNames(
     }
     return sideInputStores;
   }
+
+  public static List<Future<Void>> initAndRestoreTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, SamzaContainerMetrics samzaContainerMetrics,
+      CheckpointManager checkpointManager, JobContext jobContext, ContainerModel containerModel,
+      Map<TaskName, Checkpoint> taskCheckpoints, Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames,
+      Config config, ExecutorService executor, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      File loggerStoreDir, Map<String, SystemConsumer> storeConsumers) {
+
+    Set<String> forceRestoreTasks = new HashSet<>();
+    // Initialize each TaskStorageManager.
+    taskRestoreManagers.forEach((taskName, restoreManagers) ->
+        restoreManagers.forEach((factoryName, taskRestoreManager) -> {
+          try {
+            taskRestoreManager.init(taskCheckpoints.get(taskName));
+          } catch (SamzaException ex) {
+            if (isUnwrappedExceptionDeletedException(ex) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+              // Get deleted SnapshotIndex blob with GetDeleted and mark the task to be restored with GetDeleted as well.
+              // this ensures that the restore downloads the snapshot, recreates a new snapshot, uploads it to blob store
+              // and creates a new checkpoint.
+              ((BlobStoreRestoreManager) taskRestoreManager).init(taskCheckpoints.get(taskName), true);
+              forceRestoreTasks.add(taskName.getTaskName());
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("init failed for task: %s with GetDeleted set to true", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex);
+            }
+          }
+        })
+    );
+
+    // Start each store consumer once.
+    // Note: These consumers are per system and only changelog system store consumers will be started.
+    // Some TaskRestoreManagers may not require the consumer to be started, but due to the agnostic nature of
+    // ContainerStorageManager we always start the changelog consumer here in case it is required
+    storeConsumers.values().stream().distinct().forEach(SystemConsumer::start);
+
+    return restoreAllTaskInstances(taskRestoreManagers, taskCheckpoints, taskBackendFactoryToStoreNames, jobContext,
+        containerModel, samzaContainerMetrics, checkpointManager, config, taskInstanceMetrics, executor, loggerStoreDir,
+        forceRestoreTasks);
+  }
+
+  /**
+   * Restores all TaskInstances and returns a future for each TaskInstance restore. If this restore fails with DeletedException
+   * it will retry the restore with getDeleted flag, get all the blobs, recreate a new checkpoint in blob store and
+   * write that checkpoint to checkpoint topic.
+   * @param taskRestoreManagers map of TaskName to factory name to TaskRestoreManager map.
+   */
+  public static List<Future<Void>> restoreAllTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, Map<TaskName, Checkpoint> taskCheckpoints,
+      Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames, JobContext jobContext,
+      ContainerModel containerModel, SamzaContainerMetrics samzaContainerMetrics, CheckpointManager checkpointManager,
+      Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics, ExecutorService executor,
+      File loggedStoreDir, Set<String> forceRestoreTask) {
+
+    List<Future<Void>> taskRestoreFutures = new ArrayList<>();
+
+    // Submit restore callable for each taskInstance
+    taskRestoreManagers.forEach((taskInstanceName, restoreManagersMap) -> {
+      // Submit for each restore factory
+      restoreManagersMap.forEach((factoryName, taskRestoreManager) -> {
+        long startTime = System.currentTimeMillis();
+        String taskName = taskInstanceName.getTaskName();
+        LOG.info("Starting restore for state for task: {}", taskName);
+
+        CompletableFuture<Void> restoreFuture;
+        if (forceRestoreTask.contains(taskName) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+          restoreFuture = ((BlobStoreRestoreManager) taskRestoreManager).restore(true);
+        } else {
+          restoreFuture = taskRestoreManager.restore();
+        }
+
+        CompletableFuture<Void> taskRestoreFuture = restoreFuture.handle((res, ex) -> {
+          updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+
+          if (ex != null) {
+            if (isUnwrappedExceptionDeletedException(ex)) {
+              LOG.warn(
+                  "Restore state for task: {} received DeletedException. Reattempting with getDeletedBlobs enabled",
+                  taskInstanceName.getTaskName());
+
+              // Try to restore with getDeleted flag
+              CompletableFuture<Void> future =
+                  restoreTaskStoresAndCreateCheckpointWithGetDeleted(taskInstanceName, taskCheckpoints, checkpointManager,
+                      taskRestoreManager, config, taskInstanceMetrics, executor,
+                      taskBackendFactoryToStoreNames.get(taskInstanceName).get(factoryName), loggedStoreDir, jobContext,
+                      containerModel);
+              try {
+                if (future != null) {
+                  future.join(); // Block and restore deleted state before continuing
+                }
+              } catch (Exception e) {
+                String msg = String.format("Unable to recover from DeletedException for task %s.", taskName);
+                throw new SamzaException(msg, e);
+              } finally {
+                updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+              }
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("Error restoring state for task: %s", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex); // wrap in unchecked exception to throw from lambda
+            }
+          }
+
+          // Stop all persistent stores after restoring. Certain persistent stores opened in BulkLoad mode are compacted
+          // on stop, so paralleling stop() also parallelizes their compaction (a time-intensive operation).
+          try {
+            taskRestoreManager.close();
+          } catch (Exception e) {
+            LOG.error("Error closing restore manager for task: {} after {} restore", taskName,
+                ex != null ? "unsuccessful" : "successful", e);
+            // ignore exception from close. container may still be able to continue processing/backups
+            // if restore manager close fails.
+          }
+          return null;
+        });
+        taskRestoreFutures.add(taskRestoreFuture);
+      });
+    });
+
+    return taskRestoreFutures;
+  }
+
+  public static CompletableFuture<Void> restoreTaskStoresAndCreateCheckpointWithGetDeleted(TaskName taskName,
+      Map<TaskName, Checkpoint> taskCheckpoints, CheckpointManager checkpointManager,
+      TaskRestoreManager taskRestoreManager, Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      ExecutorService executor, Set<String> storesToRestore, File loggedStoreBaseDirectory, JobContext jobContext,
+      ContainerModel containerModel) {
+
+    // if taskInstanceMetrics are specified use those for store metrics,
+    // otherwise (in case of StorageRecovery) use a blank MetricsRegistryMap
+    MetricsRegistry metricsRegistry =
+        taskInstanceMetrics.get(taskName) != null ? taskInstanceMetrics.get(taskName).registry()
+            : new MetricsRegistryMap();
+
+    BlobStoreManager blobStoreManager = getBlobStoreManager(config, executor);
+    JobConfig jobConfig = new JobConfig(config);
+    BlobStoreUtil blobStoreUtil =
+        new BlobStoreUtil(blobStoreManager, executor, new BlobStoreConfig(config), null,
+            new BlobStoreRestoreManagerMetrics(metricsRegistry));
+
+    BlobStoreRestoreManager blobStoreRestoreManager = (BlobStoreRestoreManager) taskRestoreManager;
+
+    CheckpointId checkpointId = CheckpointId.create();
+    Map<String, String> oldSCMs = ((CheckpointV2) taskCheckpoints.get(taskName))
+        .getStateCheckpointMarkers().get(BlobStoreStateBackendFactory.class.getName());
+
+    // Returns a single future that guarantees all the following are completed, in this order:
+    // 1. Restore state locally by getting deleted blobs from the blob store.
+    // 2. Create a new Checkpoint from restored state and back it up on the blob store.
+    // 3. Clean up old/deleted Snapshot
+    // 4. Remove TTL from the new Snapshot on the blob store
+    // 5. Write the new checkpoint to checkpoint topic and update taskCheckpoints map
+
+    // 1. restore state with getDeleted flag set to true
+    return blobStoreRestoreManager.restore(true).thenCompose(r -> {
+      // 2. Create and backup new checkpoint on the blob store.
+      CompletableFuture<Map<String, String>> uploadSCMs =
+          createNewCheckpointAndBackupStores(jobContext, containerModel, config, taskName, storesToRestore, checkpointId,
+              loggedStoreBaseDirectory, blobStoreManager, metricsRegistry, executor);
+      CompletableFuture<Void> future = uploadSCMs.thenCompose(scms -> {
+        // 3. Delete prev SnapshotIndex including files/subdirs,
+        Map<String, CompletionStage<Void>> deleteOldSnapshotsFutures =
+            deletePrevSnapshots(blobStoreUtil, oldSCMs, taskName, jobConfig);
+        // 4. Mark new Snapshots to never expire
+        Map<String, CompletionStage<Void>> removeNewSnapshotsTTLFutures =
+            removeTTLNewSnapshots(blobStoreUtil, scms, taskName, jobConfig);
+        // Combined future of delete old snapshots and removeTTL of new snapshots
+        CompletableFuture<Void> removeTTLFuture = CompletableFuture.allOf(FutureUtil.mapToFuture(deleteOldSnapshotsFutures),

Review Comment:
   Should these (removeTTL, delete old snapshots) be chained instead of happening concurrently? What if removeTTL  fails but delete old snapshot completed?



##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManagerUtil.java:
##########
@@ -411,4 +438,292 @@ public static Set<String> getSideInputStoreNames(
     }
     return sideInputStores;
   }
+
+  public static List<Future<Void>> initAndRestoreTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, SamzaContainerMetrics samzaContainerMetrics,
+      CheckpointManager checkpointManager, JobContext jobContext, ContainerModel containerModel,
+      Map<TaskName, Checkpoint> taskCheckpoints, Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames,
+      Config config, ExecutorService executor, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      File loggerStoreDir, Map<String, SystemConsumer> storeConsumers) {
+
+    Set<String> forceRestoreTasks = new HashSet<>();
+    // Initialize each TaskStorageManager.
+    taskRestoreManagers.forEach((taskName, restoreManagers) ->
+        restoreManagers.forEach((factoryName, taskRestoreManager) -> {
+          try {
+            taskRestoreManager.init(taskCheckpoints.get(taskName));
+          } catch (SamzaException ex) {
+            if (isUnwrappedExceptionDeletedException(ex) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+              // Get deleted SnapshotIndex blob with GetDeleted and mark the task to be restored with GetDeleted as well.
+              // this ensures that the restore downloads the snapshot, recreates a new snapshot, uploads it to blob store
+              // and creates a new checkpoint.
+              ((BlobStoreRestoreManager) taskRestoreManager).init(taskCheckpoints.get(taskName), true);
+              forceRestoreTasks.add(taskName.getTaskName());
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("init failed for task: %s with GetDeleted set to true", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex);
+            }
+          }
+        })
+    );
+
+    // Start each store consumer once.
+    // Note: These consumers are per system and only changelog system store consumers will be started.
+    // Some TaskRestoreManagers may not require the consumer to be started, but due to the agnostic nature of
+    // ContainerStorageManager we always start the changelog consumer here in case it is required
+    storeConsumers.values().stream().distinct().forEach(SystemConsumer::start);
+
+    return restoreAllTaskInstances(taskRestoreManagers, taskCheckpoints, taskBackendFactoryToStoreNames, jobContext,
+        containerModel, samzaContainerMetrics, checkpointManager, config, taskInstanceMetrics, executor, loggerStoreDir,
+        forceRestoreTasks);
+  }
+
+  /**
+   * Restores all TaskInstances and returns a future for each TaskInstance restore. If this restore fails with DeletedException
+   * it will retry the restore with getDeleted flag, get all the blobs, recreate a new checkpoint in blob store and
+   * write that checkpoint to checkpoint topic.
+   * @param taskRestoreManagers map of TaskName to factory name to TaskRestoreManager map.
+   */
+  public static List<Future<Void>> restoreAllTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, Map<TaskName, Checkpoint> taskCheckpoints,
+      Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames, JobContext jobContext,
+      ContainerModel containerModel, SamzaContainerMetrics samzaContainerMetrics, CheckpointManager checkpointManager,
+      Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics, ExecutorService executor,
+      File loggedStoreDir, Set<String> forceRestoreTask) {
+
+    List<Future<Void>> taskRestoreFutures = new ArrayList<>();
+
+    // Submit restore callable for each taskInstance
+    taskRestoreManagers.forEach((taskInstanceName, restoreManagersMap) -> {
+      // Submit for each restore factory
+      restoreManagersMap.forEach((factoryName, taskRestoreManager) -> {
+        long startTime = System.currentTimeMillis();
+        String taskName = taskInstanceName.getTaskName();
+        LOG.info("Starting restore for state for task: {}", taskName);
+
+        CompletableFuture<Void> restoreFuture;
+        if (forceRestoreTask.contains(taskName) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+          restoreFuture = ((BlobStoreRestoreManager) taskRestoreManager).restore(true);
+        } else {
+          restoreFuture = taskRestoreManager.restore();
+        }
+
+        CompletableFuture<Void> taskRestoreFuture = restoreFuture.handle((res, ex) -> {
+          updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+
+          if (ex != null) {
+            if (isUnwrappedExceptionDeletedException(ex)) {
+              LOG.warn(
+                  "Restore state for task: {} received DeletedException. Reattempting with getDeletedBlobs enabled",
+                  taskInstanceName.getTaskName());
+
+              // Try to restore with getDeleted flag
+              CompletableFuture<Void> future =
+                  restoreTaskStoresAndCreateCheckpointWithGetDeleted(taskInstanceName, taskCheckpoints, checkpointManager,
+                      taskRestoreManager, config, taskInstanceMetrics, executor,
+                      taskBackendFactoryToStoreNames.get(taskInstanceName).get(factoryName), loggedStoreDir, jobContext,
+                      containerModel);
+              try {
+                if (future != null) {
+                  future.join(); // Block and restore deleted state before continuing
+                }
+              } catch (Exception e) {
+                String msg = String.format("Unable to recover from DeletedException for task %s.", taskName);
+                throw new SamzaException(msg, e);
+              } finally {
+                updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+              }
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("Error restoring state for task: %s", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex); // wrap in unchecked exception to throw from lambda
+            }
+          }
+
+          // Stop all persistent stores after restoring. Certain persistent stores opened in BulkLoad mode are compacted
+          // on stop, so paralleling stop() also parallelizes their compaction (a time-intensive operation).
+          try {
+            taskRestoreManager.close();
+          } catch (Exception e) {
+            LOG.error("Error closing restore manager for task: {} after {} restore", taskName,
+                ex != null ? "unsuccessful" : "successful", e);
+            // ignore exception from close. container may still be able to continue processing/backups
+            // if restore manager close fails.
+          }
+          return null;
+        });
+        taskRestoreFutures.add(taskRestoreFuture);
+      });
+    });
+
+    return taskRestoreFutures;
+  }
+
+  public static CompletableFuture<Void> restoreTaskStoresAndCreateCheckpointWithGetDeleted(TaskName taskName,
+      Map<TaskName, Checkpoint> taskCheckpoints, CheckpointManager checkpointManager,
+      TaskRestoreManager taskRestoreManager, Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      ExecutorService executor, Set<String> storesToRestore, File loggedStoreBaseDirectory, JobContext jobContext,
+      ContainerModel containerModel) {
+
+    // if taskInstanceMetrics are specified use those for store metrics,
+    // otherwise (in case of StorageRecovery) use a blank MetricsRegistryMap
+    MetricsRegistry metricsRegistry =
+        taskInstanceMetrics.get(taskName) != null ? taskInstanceMetrics.get(taskName).registry()
+            : new MetricsRegistryMap();
+
+    BlobStoreManager blobStoreManager = getBlobStoreManager(config, executor);
+    JobConfig jobConfig = new JobConfig(config);
+    BlobStoreUtil blobStoreUtil =
+        new BlobStoreUtil(blobStoreManager, executor, new BlobStoreConfig(config), null,
+            new BlobStoreRestoreManagerMetrics(metricsRegistry));
+
+    BlobStoreRestoreManager blobStoreRestoreManager = (BlobStoreRestoreManager) taskRestoreManager;
+
+    CheckpointId checkpointId = CheckpointId.create();
+    Map<String, String> oldSCMs = ((CheckpointV2) taskCheckpoints.get(taskName))
+        .getStateCheckpointMarkers().get(BlobStoreStateBackendFactory.class.getName());
+
+    // Returns a single future that guarantees all the following are completed, in this order:
+    // 1. Restore state locally by getting deleted blobs from the blob store.
+    // 2. Create a new Checkpoint from restored state and back it up on the blob store.

Review Comment:
   Snapshot, not checkpoint.



##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManagerUtil.java:
##########
@@ -411,4 +438,292 @@ public static Set<String> getSideInputStoreNames(
     }
     return sideInputStores;
   }
+
+  public static List<Future<Void>> initAndRestoreTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, SamzaContainerMetrics samzaContainerMetrics,
+      CheckpointManager checkpointManager, JobContext jobContext, ContainerModel containerModel,
+      Map<TaskName, Checkpoint> taskCheckpoints, Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames,
+      Config config, ExecutorService executor, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      File loggerStoreDir, Map<String, SystemConsumer> storeConsumers) {
+
+    Set<String> forceRestoreTasks = new HashSet<>();
+    // Initialize each TaskStorageManager.
+    taskRestoreManagers.forEach((taskName, restoreManagers) ->
+        restoreManagers.forEach((factoryName, taskRestoreManager) -> {
+          try {
+            taskRestoreManager.init(taskCheckpoints.get(taskName));
+          } catch (SamzaException ex) {
+            if (isUnwrappedExceptionDeletedException(ex) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+              // Get deleted SnapshotIndex blob with GetDeleted and mark the task to be restored with GetDeleted as well.
+              // this ensures that the restore downloads the snapshot, recreates a new snapshot, uploads it to blob store
+              // and creates a new checkpoint.
+              ((BlobStoreRestoreManager) taskRestoreManager).init(taskCheckpoints.get(taskName), true);
+              forceRestoreTasks.add(taskName.getTaskName());
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("init failed for task: %s with GetDeleted set to true", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex);
+            }
+          }
+        })
+    );
+
+    // Start each store consumer once.
+    // Note: These consumers are per system and only changelog system store consumers will be started.
+    // Some TaskRestoreManagers may not require the consumer to be started, but due to the agnostic nature of
+    // ContainerStorageManager we always start the changelog consumer here in case it is required
+    storeConsumers.values().stream().distinct().forEach(SystemConsumer::start);
+
+    return restoreAllTaskInstances(taskRestoreManagers, taskCheckpoints, taskBackendFactoryToStoreNames, jobContext,
+        containerModel, samzaContainerMetrics, checkpointManager, config, taskInstanceMetrics, executor, loggerStoreDir,
+        forceRestoreTasks);
+  }
+
+  /**
+   * Restores all TaskInstances and returns a future for each TaskInstance restore. If this restore fails with DeletedException
+   * it will retry the restore with getDeleted flag, get all the blobs, recreate a new checkpoint in blob store and
+   * write that checkpoint to checkpoint topic.
+   * @param taskRestoreManagers map of TaskName to factory name to TaskRestoreManager map.
+   */
+  public static List<Future<Void>> restoreAllTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, Map<TaskName, Checkpoint> taskCheckpoints,
+      Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames, JobContext jobContext,
+      ContainerModel containerModel, SamzaContainerMetrics samzaContainerMetrics, CheckpointManager checkpointManager,
+      Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics, ExecutorService executor,
+      File loggedStoreDir, Set<String> forceRestoreTask) {
+
+    List<Future<Void>> taskRestoreFutures = new ArrayList<>();
+
+    // Submit restore callable for each taskInstance
+    taskRestoreManagers.forEach((taskInstanceName, restoreManagersMap) -> {
+      // Submit for each restore factory
+      restoreManagersMap.forEach((factoryName, taskRestoreManager) -> {
+        long startTime = System.currentTimeMillis();
+        String taskName = taskInstanceName.getTaskName();
+        LOG.info("Starting restore for state for task: {}", taskName);
+
+        CompletableFuture<Void> restoreFuture;
+        if (forceRestoreTask.contains(taskName) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+          restoreFuture = ((BlobStoreRestoreManager) taskRestoreManager).restore(true);
+        } else {
+          restoreFuture = taskRestoreManager.restore();
+        }
+
+        CompletableFuture<Void> taskRestoreFuture = restoreFuture.handle((res, ex) -> {
+          updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+
+          if (ex != null) {
+            if (isUnwrappedExceptionDeletedException(ex)) {
+              LOG.warn(
+                  "Restore state for task: {} received DeletedException. Reattempting with getDeletedBlobs enabled",
+                  taskInstanceName.getTaskName());
+
+              // Try to restore with getDeleted flag
+              CompletableFuture<Void> future =
+                  restoreTaskStoresAndCreateCheckpointWithGetDeleted(taskInstanceName, taskCheckpoints, checkpointManager,
+                      taskRestoreManager, config, taskInstanceMetrics, executor,
+                      taskBackendFactoryToStoreNames.get(taskInstanceName).get(factoryName), loggedStoreDir, jobContext,
+                      containerModel);
+              try {
+                if (future != null) {
+                  future.join(); // Block and restore deleted state before continuing
+                }
+              } catch (Exception e) {
+                String msg = String.format("Unable to recover from DeletedException for task %s.", taskName);
+                throw new SamzaException(msg, e);
+              } finally {
+                updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+              }
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("Error restoring state for task: %s", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex); // wrap in unchecked exception to throw from lambda
+            }
+          }
+
+          // Stop all persistent stores after restoring. Certain persistent stores opened in BulkLoad mode are compacted
+          // on stop, so paralleling stop() also parallelizes their compaction (a time-intensive operation).
+          try {
+            taskRestoreManager.close();
+          } catch (Exception e) {
+            LOG.error("Error closing restore manager for task: {} after {} restore", taskName,
+                ex != null ? "unsuccessful" : "successful", e);
+            // ignore exception from close. container may still be able to continue processing/backups
+            // if restore manager close fails.
+          }
+          return null;
+        });
+        taskRestoreFutures.add(taskRestoreFuture);
+      });
+    });
+
+    return taskRestoreFutures;
+  }
+
+  public static CompletableFuture<Void> restoreTaskStoresAndCreateCheckpointWithGetDeleted(TaskName taskName,
+      Map<TaskName, Checkpoint> taskCheckpoints, CheckpointManager checkpointManager,
+      TaskRestoreManager taskRestoreManager, Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      ExecutorService executor, Set<String> storesToRestore, File loggedStoreBaseDirectory, JobContext jobContext,
+      ContainerModel containerModel) {
+
+    // if taskInstanceMetrics are specified use those for store metrics,
+    // otherwise (in case of StorageRecovery) use a blank MetricsRegistryMap
+    MetricsRegistry metricsRegistry =
+        taskInstanceMetrics.get(taskName) != null ? taskInstanceMetrics.get(taskName).registry()
+            : new MetricsRegistryMap();
+
+    BlobStoreManager blobStoreManager = getBlobStoreManager(config, executor);
+    JobConfig jobConfig = new JobConfig(config);
+    BlobStoreUtil blobStoreUtil =
+        new BlobStoreUtil(blobStoreManager, executor, new BlobStoreConfig(config), null,
+            new BlobStoreRestoreManagerMetrics(metricsRegistry));
+
+    BlobStoreRestoreManager blobStoreRestoreManager = (BlobStoreRestoreManager) taskRestoreManager;
+
+    CheckpointId checkpointId = CheckpointId.create();
+    Map<String, String> oldSCMs = ((CheckpointV2) taskCheckpoints.get(taskName))
+        .getStateCheckpointMarkers().get(BlobStoreStateBackendFactory.class.getName());
+
+    // Returns a single future that guarantees all the following are completed, in this order:
+    // 1. Restore state locally by getting deleted blobs from the blob store.
+    // 2. Create a new Checkpoint from restored state and back it up on the blob store.
+    // 3. Clean up old/deleted Snapshot
+    // 4. Remove TTL from the new Snapshot on the blob store
+    // 5. Write the new checkpoint to checkpoint topic and update taskCheckpoints map
+
+    // 1. restore state with getDeleted flag set to true
+    return blobStoreRestoreManager.restore(true).thenCompose(r -> {
+      // 2. Create and backup new checkpoint on the blob store.
+      CompletableFuture<Map<String, String>> uploadSCMs =
+          createNewCheckpointAndBackupStores(jobContext, containerModel, config, taskName, storesToRestore, checkpointId,
+              loggedStoreBaseDirectory, blobStoreManager, metricsRegistry, executor);
+      CompletableFuture<Void> future = uploadSCMs.thenCompose(scms -> {
+        // 3. Delete prev SnapshotIndex including files/subdirs,
+        Map<String, CompletionStage<Void>> deleteOldSnapshotsFutures =
+            deletePrevSnapshots(blobStoreUtil, oldSCMs, taskName, jobConfig);
+        // 4. Mark new Snapshots to never expire
+        Map<String, CompletionStage<Void>> removeNewSnapshotsTTLFutures =
+            removeTTLNewSnapshots(blobStoreUtil, scms, taskName, jobConfig);
+        // Combined future of delete old snapshots and removeTTL of new snapshots
+        CompletableFuture<Void> removeTTLFuture = CompletableFuture.allOf(FutureUtil.mapToFuture(deleteOldSnapshotsFutures),
+            FutureUtil.mapToFuture(removeNewSnapshotsTTLFutures));
+        // 5. Update taskCheckpoints map with new checkpoint
+        CompletableFuture<Void> checkpointFuture = removeTTLFuture.thenRun(() -> {
+          Checkpoint newCheckpoint = writeNewCheckpoint(taskName, checkpointId, scms, checkpointManager);
+          taskCheckpoints.put(taskName, newCheckpoint);
+        });
+        return checkpointFuture;
+      });
+      return future;
+    }).exceptionally(ex -> {
+      String msg = String.format("Restore for task %s failed with get deleted bob", taskName);
+      throw new SamzaException(msg, ex);
+    });
+  }
+
+  private static CompletableFuture<Map<String, String>> createNewCheckpointAndBackupStores(JobContext jobContext,

Review Comment:
   Creating checkpoint dir is impl detail of backing up stores. Is not relevant to rest of the flow and should not be present in method name.



##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManagerUtil.java:
##########
@@ -411,4 +438,292 @@ public static Set<String> getSideInputStoreNames(
     }
     return sideInputStores;
   }
+
+  public static List<Future<Void>> initAndRestoreTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, SamzaContainerMetrics samzaContainerMetrics,
+      CheckpointManager checkpointManager, JobContext jobContext, ContainerModel containerModel,
+      Map<TaskName, Checkpoint> taskCheckpoints, Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames,
+      Config config, ExecutorService executor, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      File loggerStoreDir, Map<String, SystemConsumer> storeConsumers) {
+
+    Set<String> forceRestoreTasks = new HashSet<>();
+    // Initialize each TaskStorageManager.
+    taskRestoreManagers.forEach((taskName, restoreManagers) ->
+        restoreManagers.forEach((factoryName, taskRestoreManager) -> {
+          try {
+            taskRestoreManager.init(taskCheckpoints.get(taskName));
+          } catch (SamzaException ex) {
+            if (isUnwrappedExceptionDeletedException(ex) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+              // Get deleted SnapshotIndex blob with GetDeleted and mark the task to be restored with GetDeleted as well.
+              // this ensures that the restore downloads the snapshot, recreates a new snapshot, uploads it to blob store
+              // and creates a new checkpoint.
+              ((BlobStoreRestoreManager) taskRestoreManager).init(taskCheckpoints.get(taskName), true);
+              forceRestoreTasks.add(taskName.getTaskName());
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("init failed for task: %s with GetDeleted set to true", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex);
+            }
+          }
+        })
+    );
+
+    // Start each store consumer once.
+    // Note: These consumers are per system and only changelog system store consumers will be started.
+    // Some TaskRestoreManagers may not require the consumer to be started, but due to the agnostic nature of
+    // ContainerStorageManager we always start the changelog consumer here in case it is required
+    storeConsumers.values().stream().distinct().forEach(SystemConsumer::start);
+
+    return restoreAllTaskInstances(taskRestoreManagers, taskCheckpoints, taskBackendFactoryToStoreNames, jobContext,
+        containerModel, samzaContainerMetrics, checkpointManager, config, taskInstanceMetrics, executor, loggerStoreDir,
+        forceRestoreTasks);
+  }
+
+  /**
+   * Restores all TaskInstances and returns a future for each TaskInstance restore. If this restore fails with DeletedException
+   * it will retry the restore with getDeleted flag, get all the blobs, recreate a new checkpoint in blob store and
+   * write that checkpoint to checkpoint topic.
+   * @param taskRestoreManagers map of TaskName to factory name to TaskRestoreManager map.
+   */
+  public static List<Future<Void>> restoreAllTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, Map<TaskName, Checkpoint> taskCheckpoints,
+      Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames, JobContext jobContext,
+      ContainerModel containerModel, SamzaContainerMetrics samzaContainerMetrics, CheckpointManager checkpointManager,
+      Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics, ExecutorService executor,
+      File loggedStoreDir, Set<String> forceRestoreTask) {
+
+    List<Future<Void>> taskRestoreFutures = new ArrayList<>();
+
+    // Submit restore callable for each taskInstance
+    taskRestoreManagers.forEach((taskInstanceName, restoreManagersMap) -> {
+      // Submit for each restore factory
+      restoreManagersMap.forEach((factoryName, taskRestoreManager) -> {
+        long startTime = System.currentTimeMillis();
+        String taskName = taskInstanceName.getTaskName();
+        LOG.info("Starting restore for state for task: {}", taskName);
+
+        CompletableFuture<Void> restoreFuture;
+        if (forceRestoreTask.contains(taskName) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+          restoreFuture = ((BlobStoreRestoreManager) taskRestoreManager).restore(true);
+        } else {
+          restoreFuture = taskRestoreManager.restore();
+        }
+
+        CompletableFuture<Void> taskRestoreFuture = restoreFuture.handle((res, ex) -> {
+          updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+
+          if (ex != null) {
+            if (isUnwrappedExceptionDeletedException(ex)) {
+              LOG.warn(
+                  "Restore state for task: {} received DeletedException. Reattempting with getDeletedBlobs enabled",
+                  taskInstanceName.getTaskName());
+
+              // Try to restore with getDeleted flag
+              CompletableFuture<Void> future =
+                  restoreTaskStoresAndCreateCheckpointWithGetDeleted(taskInstanceName, taskCheckpoints, checkpointManager,
+                      taskRestoreManager, config, taskInstanceMetrics, executor,
+                      taskBackendFactoryToStoreNames.get(taskInstanceName).get(factoryName), loggedStoreDir, jobContext,
+                      containerModel);
+              try {
+                if (future != null) {
+                  future.join(); // Block and restore deleted state before continuing
+                }
+              } catch (Exception e) {
+                String msg = String.format("Unable to recover from DeletedException for task %s.", taskName);
+                throw new SamzaException(msg, e);
+              } finally {
+                updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+              }
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("Error restoring state for task: %s", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex); // wrap in unchecked exception to throw from lambda
+            }
+          }
+
+          // Stop all persistent stores after restoring. Certain persistent stores opened in BulkLoad mode are compacted
+          // on stop, so paralleling stop() also parallelizes their compaction (a time-intensive operation).
+          try {
+            taskRestoreManager.close();
+          } catch (Exception e) {
+            LOG.error("Error closing restore manager for task: {} after {} restore", taskName,
+                ex != null ? "unsuccessful" : "successful", e);
+            // ignore exception from close. container may still be able to continue processing/backups
+            // if restore manager close fails.
+          }
+          return null;
+        });
+        taskRestoreFutures.add(taskRestoreFuture);
+      });
+    });
+
+    return taskRestoreFutures;
+  }
+
+  public static CompletableFuture<Void> restoreTaskStoresAndCreateCheckpointWithGetDeleted(TaskName taskName,
+      Map<TaskName, Checkpoint> taskCheckpoints, CheckpointManager checkpointManager,
+      TaskRestoreManager taskRestoreManager, Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      ExecutorService executor, Set<String> storesToRestore, File loggedStoreBaseDirectory, JobContext jobContext,
+      ContainerModel containerModel) {
+
+    // if taskInstanceMetrics are specified use those for store metrics,
+    // otherwise (in case of StorageRecovery) use a blank MetricsRegistryMap
+    MetricsRegistry metricsRegistry =
+        taskInstanceMetrics.get(taskName) != null ? taskInstanceMetrics.get(taskName).registry()
+            : new MetricsRegistryMap();
+
+    BlobStoreManager blobStoreManager = getBlobStoreManager(config, executor);
+    JobConfig jobConfig = new JobConfig(config);
+    BlobStoreUtil blobStoreUtil =
+        new BlobStoreUtil(blobStoreManager, executor, new BlobStoreConfig(config), null,
+            new BlobStoreRestoreManagerMetrics(metricsRegistry));
+
+    BlobStoreRestoreManager blobStoreRestoreManager = (BlobStoreRestoreManager) taskRestoreManager;
+
+    CheckpointId checkpointId = CheckpointId.create();
+    Map<String, String> oldSCMs = ((CheckpointV2) taskCheckpoints.get(taskName))
+        .getStateCheckpointMarkers().get(BlobStoreStateBackendFactory.class.getName());
+
+    // Returns a single future that guarantees all the following are completed, in this order:
+    // 1. Restore state locally by getting deleted blobs from the blob store.
+    // 2. Create a new Checkpoint from restored state and back it up on the blob store.
+    // 3. Clean up old/deleted Snapshot
+    // 4. Remove TTL from the new Snapshot on the blob store

Review Comment:
   "... and blobs"?



##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManagerUtil.java:
##########
@@ -411,4 +438,292 @@ public static Set<String> getSideInputStoreNames(
     }
     return sideInputStores;
   }
+
+  public static List<Future<Void>> initAndRestoreTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, SamzaContainerMetrics samzaContainerMetrics,
+      CheckpointManager checkpointManager, JobContext jobContext, ContainerModel containerModel,
+      Map<TaskName, Checkpoint> taskCheckpoints, Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames,
+      Config config, ExecutorService executor, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      File loggerStoreDir, Map<String, SystemConsumer> storeConsumers) {
+
+    Set<String> forceRestoreTasks = new HashSet<>();
+    // Initialize each TaskStorageManager.
+    taskRestoreManagers.forEach((taskName, restoreManagers) ->
+        restoreManagers.forEach((factoryName, taskRestoreManager) -> {
+          try {
+            taskRestoreManager.init(taskCheckpoints.get(taskName));
+          } catch (SamzaException ex) {
+            if (isUnwrappedExceptionDeletedException(ex) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+              // Get deleted SnapshotIndex blob with GetDeleted and mark the task to be restored with GetDeleted as well.
+              // this ensures that the restore downloads the snapshot, recreates a new snapshot, uploads it to blob store
+              // and creates a new checkpoint.
+              ((BlobStoreRestoreManager) taskRestoreManager).init(taskCheckpoints.get(taskName), true);
+              forceRestoreTasks.add(taskName.getTaskName());
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("init failed for task: %s with GetDeleted set to true", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex);
+            }
+          }
+        })
+    );
+
+    // Start each store consumer once.
+    // Note: These consumers are per system and only changelog system store consumers will be started.
+    // Some TaskRestoreManagers may not require the consumer to be started, but due to the agnostic nature of
+    // ContainerStorageManager we always start the changelog consumer here in case it is required
+    storeConsumers.values().stream().distinct().forEach(SystemConsumer::start);
+
+    return restoreAllTaskInstances(taskRestoreManagers, taskCheckpoints, taskBackendFactoryToStoreNames, jobContext,
+        containerModel, samzaContainerMetrics, checkpointManager, config, taskInstanceMetrics, executor, loggerStoreDir,
+        forceRestoreTasks);
+  }
+
+  /**
+   * Restores all TaskInstances and returns a future for each TaskInstance restore. If this restore fails with DeletedException
+   * it will retry the restore with getDeleted flag, get all the blobs, recreate a new checkpoint in blob store and
+   * write that checkpoint to checkpoint topic.
+   * @param taskRestoreManagers map of TaskName to factory name to TaskRestoreManager map.
+   */
+  public static List<Future<Void>> restoreAllTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, Map<TaskName, Checkpoint> taskCheckpoints,
+      Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames, JobContext jobContext,
+      ContainerModel containerModel, SamzaContainerMetrics samzaContainerMetrics, CheckpointManager checkpointManager,
+      Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics, ExecutorService executor,
+      File loggedStoreDir, Set<String> forceRestoreTask) {
+
+    List<Future<Void>> taskRestoreFutures = new ArrayList<>();
+
+    // Submit restore callable for each taskInstance
+    taskRestoreManagers.forEach((taskInstanceName, restoreManagersMap) -> {
+      // Submit for each restore factory
+      restoreManagersMap.forEach((factoryName, taskRestoreManager) -> {
+        long startTime = System.currentTimeMillis();
+        String taskName = taskInstanceName.getTaskName();
+        LOG.info("Starting restore for state for task: {}", taskName);
+
+        CompletableFuture<Void> restoreFuture;
+        if (forceRestoreTask.contains(taskName) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+          restoreFuture = ((BlobStoreRestoreManager) taskRestoreManager).restore(true);
+        } else {
+          restoreFuture = taskRestoreManager.restore();
+        }
+
+        CompletableFuture<Void> taskRestoreFuture = restoreFuture.handle((res, ex) -> {
+          updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+
+          if (ex != null) {
+            if (isUnwrappedExceptionDeletedException(ex)) {
+              LOG.warn(
+                  "Restore state for task: {} received DeletedException. Reattempting with getDeletedBlobs enabled",
+                  taskInstanceName.getTaskName());
+
+              // Try to restore with getDeleted flag
+              CompletableFuture<Void> future =
+                  restoreTaskStoresAndCreateCheckpointWithGetDeleted(taskInstanceName, taskCheckpoints, checkpointManager,
+                      taskRestoreManager, config, taskInstanceMetrics, executor,
+                      taskBackendFactoryToStoreNames.get(taskInstanceName).get(factoryName), loggedStoreDir, jobContext,
+                      containerModel);
+              try {
+                if (future != null) {
+                  future.join(); // Block and restore deleted state before continuing
+                }
+              } catch (Exception e) {
+                String msg = String.format("Unable to recover from DeletedException for task %s.", taskName);
+                throw new SamzaException(msg, e);
+              } finally {
+                updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+              }
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("Error restoring state for task: %s", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex); // wrap in unchecked exception to throw from lambda
+            }
+          }
+
+          // Stop all persistent stores after restoring. Certain persistent stores opened in BulkLoad mode are compacted
+          // on stop, so paralleling stop() also parallelizes their compaction (a time-intensive operation).
+          try {
+            taskRestoreManager.close();
+          } catch (Exception e) {
+            LOG.error("Error closing restore manager for task: {} after {} restore", taskName,
+                ex != null ? "unsuccessful" : "successful", e);
+            // ignore exception from close. container may still be able to continue processing/backups
+            // if restore manager close fails.
+          }
+          return null;
+        });
+        taskRestoreFutures.add(taskRestoreFuture);
+      });
+    });
+
+    return taskRestoreFutures;
+  }
+
+  public static CompletableFuture<Void> restoreTaskStoresAndCreateCheckpointWithGetDeleted(TaskName taskName,
+      Map<TaskName, Checkpoint> taskCheckpoints, CheckpointManager checkpointManager,
+      TaskRestoreManager taskRestoreManager, Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      ExecutorService executor, Set<String> storesToRestore, File loggedStoreBaseDirectory, JobContext jobContext,
+      ContainerModel containerModel) {
+
+    // if taskInstanceMetrics are specified use those for store metrics,
+    // otherwise (in case of StorageRecovery) use a blank MetricsRegistryMap
+    MetricsRegistry metricsRegistry =
+        taskInstanceMetrics.get(taskName) != null ? taskInstanceMetrics.get(taskName).registry()
+            : new MetricsRegistryMap();
+
+    BlobStoreManager blobStoreManager = getBlobStoreManager(config, executor);
+    JobConfig jobConfig = new JobConfig(config);
+    BlobStoreUtil blobStoreUtil =
+        new BlobStoreUtil(blobStoreManager, executor, new BlobStoreConfig(config), null,
+            new BlobStoreRestoreManagerMetrics(metricsRegistry));
+
+    BlobStoreRestoreManager blobStoreRestoreManager = (BlobStoreRestoreManager) taskRestoreManager;
+
+    CheckpointId checkpointId = CheckpointId.create();
+    Map<String, String> oldSCMs = ((CheckpointV2) taskCheckpoints.get(taskName))
+        .getStateCheckpointMarkers().get(BlobStoreStateBackendFactory.class.getName());
+
+    // Returns a single future that guarantees all the following are completed, in this order:
+    // 1. Restore state locally by getting deleted blobs from the blob store.
+    // 2. Create a new Checkpoint from restored state and back it up on the blob store.
+    // 3. Clean up old/deleted Snapshot
+    // 4. Remove TTL from the new Snapshot on the blob store
+    // 5. Write the new checkpoint to checkpoint topic and update taskCheckpoints map
+
+    // 1. restore state with getDeleted flag set to true
+    return blobStoreRestoreManager.restore(true).thenCompose(r -> {
+      // 2. Create and backup new checkpoint on the blob store.
+      CompletableFuture<Map<String, String>> uploadSCMs =
+          createNewCheckpointAndBackupStores(jobContext, containerModel, config, taskName, storesToRestore, checkpointId,

Review Comment:
   See comment below re: method name.



##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManagerUtil.java:
##########
@@ -411,4 +438,292 @@ public static Set<String> getSideInputStoreNames(
     }
     return sideInputStores;
   }
+
+  public static List<Future<Void>> initAndRestoreTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, SamzaContainerMetrics samzaContainerMetrics,
+      CheckpointManager checkpointManager, JobContext jobContext, ContainerModel containerModel,
+      Map<TaskName, Checkpoint> taskCheckpoints, Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames,
+      Config config, ExecutorService executor, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      File loggerStoreDir, Map<String, SystemConsumer> storeConsumers) {
+
+    Set<String> forceRestoreTasks = new HashSet<>();
+    // Initialize each TaskStorageManager.
+    taskRestoreManagers.forEach((taskName, restoreManagers) ->
+        restoreManagers.forEach((factoryName, taskRestoreManager) -> {
+          try {
+            taskRestoreManager.init(taskCheckpoints.get(taskName));
+          } catch (SamzaException ex) {
+            if (isUnwrappedExceptionDeletedException(ex) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+              // Get deleted SnapshotIndex blob with GetDeleted and mark the task to be restored with GetDeleted as well.
+              // this ensures that the restore downloads the snapshot, recreates a new snapshot, uploads it to blob store
+              // and creates a new checkpoint.
+              ((BlobStoreRestoreManager) taskRestoreManager).init(taskCheckpoints.get(taskName), true);
+              forceRestoreTasks.add(taskName.getTaskName());
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("init failed for task: %s with GetDeleted set to true", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex);
+            }
+          }
+        })
+    );
+
+    // Start each store consumer once.
+    // Note: These consumers are per system and only changelog system store consumers will be started.
+    // Some TaskRestoreManagers may not require the consumer to be started, but due to the agnostic nature of
+    // ContainerStorageManager we always start the changelog consumer here in case it is required
+    storeConsumers.values().stream().distinct().forEach(SystemConsumer::start);
+
+    return restoreAllTaskInstances(taskRestoreManagers, taskCheckpoints, taskBackendFactoryToStoreNames, jobContext,
+        containerModel, samzaContainerMetrics, checkpointManager, config, taskInstanceMetrics, executor, loggerStoreDir,
+        forceRestoreTasks);
+  }
+
+  /**
+   * Restores all TaskInstances and returns a future for each TaskInstance restore. If this restore fails with DeletedException
+   * it will retry the restore with getDeleted flag, get all the blobs, recreate a new checkpoint in blob store and
+   * write that checkpoint to checkpoint topic.
+   * @param taskRestoreManagers map of TaskName to factory name to TaskRestoreManager map.
+   */
+  public static List<Future<Void>> restoreAllTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, Map<TaskName, Checkpoint> taskCheckpoints,
+      Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames, JobContext jobContext,
+      ContainerModel containerModel, SamzaContainerMetrics samzaContainerMetrics, CheckpointManager checkpointManager,
+      Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics, ExecutorService executor,
+      File loggedStoreDir, Set<String> forceRestoreTask) {
+
+    List<Future<Void>> taskRestoreFutures = new ArrayList<>();
+
+    // Submit restore callable for each taskInstance
+    taskRestoreManagers.forEach((taskInstanceName, restoreManagersMap) -> {
+      // Submit for each restore factory
+      restoreManagersMap.forEach((factoryName, taskRestoreManager) -> {
+        long startTime = System.currentTimeMillis();
+        String taskName = taskInstanceName.getTaskName();
+        LOG.info("Starting restore for state for task: {}", taskName);
+
+        CompletableFuture<Void> restoreFuture;
+        if (forceRestoreTask.contains(taskName) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+          restoreFuture = ((BlobStoreRestoreManager) taskRestoreManager).restore(true);
+        } else {
+          restoreFuture = taskRestoreManager.restore();
+        }
+
+        CompletableFuture<Void> taskRestoreFuture = restoreFuture.handle((res, ex) -> {
+          updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+
+          if (ex != null) {
+            if (isUnwrappedExceptionDeletedException(ex)) {
+              LOG.warn(
+                  "Restore state for task: {} received DeletedException. Reattempting with getDeletedBlobs enabled",
+                  taskInstanceName.getTaskName());
+
+              // Try to restore with getDeleted flag
+              CompletableFuture<Void> future =
+                  restoreTaskStoresAndCreateCheckpointWithGetDeleted(taskInstanceName, taskCheckpoints, checkpointManager,
+                      taskRestoreManager, config, taskInstanceMetrics, executor,
+                      taskBackendFactoryToStoreNames.get(taskInstanceName).get(factoryName), loggedStoreDir, jobContext,
+                      containerModel);
+              try {
+                if (future != null) {
+                  future.join(); // Block and restore deleted state before continuing
+                }
+              } catch (Exception e) {
+                String msg = String.format("Unable to recover from DeletedException for task %s.", taskName);
+                throw new SamzaException(msg, e);
+              } finally {
+                updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+              }
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("Error restoring state for task: %s", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex); // wrap in unchecked exception to throw from lambda
+            }
+          }
+
+          // Stop all persistent stores after restoring. Certain persistent stores opened in BulkLoad mode are compacted
+          // on stop, so paralleling stop() also parallelizes their compaction (a time-intensive operation).
+          try {
+            taskRestoreManager.close();
+          } catch (Exception e) {
+            LOG.error("Error closing restore manager for task: {} after {} restore", taskName,
+                ex != null ? "unsuccessful" : "successful", e);
+            // ignore exception from close. container may still be able to continue processing/backups
+            // if restore manager close fails.
+          }
+          return null;
+        });
+        taskRestoreFutures.add(taskRestoreFuture);
+      });
+    });
+
+    return taskRestoreFutures;
+  }
+
+  public static CompletableFuture<Void> restoreTaskStoresAndCreateCheckpointWithGetDeleted(TaskName taskName,
+      Map<TaskName, Checkpoint> taskCheckpoints, CheckpointManager checkpointManager,
+      TaskRestoreManager taskRestoreManager, Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      ExecutorService executor, Set<String> storesToRestore, File loggedStoreBaseDirectory, JobContext jobContext,
+      ContainerModel containerModel) {
+
+    // if taskInstanceMetrics are specified use those for store metrics,
+    // otherwise (in case of StorageRecovery) use a blank MetricsRegistryMap
+    MetricsRegistry metricsRegistry =
+        taskInstanceMetrics.get(taskName) != null ? taskInstanceMetrics.get(taskName).registry()
+            : new MetricsRegistryMap();
+
+    BlobStoreManager blobStoreManager = getBlobStoreManager(config, executor);
+    JobConfig jobConfig = new JobConfig(config);
+    BlobStoreUtil blobStoreUtil =
+        new BlobStoreUtil(blobStoreManager, executor, new BlobStoreConfig(config), null,
+            new BlobStoreRestoreManagerMetrics(metricsRegistry));
+
+    BlobStoreRestoreManager blobStoreRestoreManager = (BlobStoreRestoreManager) taskRestoreManager;
+
+    CheckpointId checkpointId = CheckpointId.create();
+    Map<String, String> oldSCMs = ((CheckpointV2) taskCheckpoints.get(taskName))
+        .getStateCheckpointMarkers().get(BlobStoreStateBackendFactory.class.getName());
+
+    // Returns a single future that guarantees all the following are completed, in this order:
+    // 1. Restore state locally by getting deleted blobs from the blob store.
+    // 2. Create a new Checkpoint from restored state and back it up on the blob store.
+    // 3. Clean up old/deleted Snapshot
+    // 4. Remove TTL from the new Snapshot on the blob store
+    // 5. Write the new checkpoint to checkpoint topic and update taskCheckpoints map
+
+    // 1. restore state with getDeleted flag set to true
+    return blobStoreRestoreManager.restore(true).thenCompose(r -> {
+      // 2. Create and backup new checkpoint on the blob store.
+      CompletableFuture<Map<String, String>> uploadSCMs =
+          createNewCheckpointAndBackupStores(jobContext, containerModel, config, taskName, storesToRestore, checkpointId,
+              loggedStoreBaseDirectory, blobStoreManager, metricsRegistry, executor);
+      CompletableFuture<Void> future = uploadSCMs.thenCompose(scms -> {
+        // 3. Delete prev SnapshotIndex including files/subdirs,
+        Map<String, CompletionStage<Void>> deleteOldSnapshotsFutures =
+            deletePrevSnapshots(blobStoreUtil, oldSCMs, taskName, jobConfig);
+        // 4. Mark new Snapshots to never expire

Review Comment:
   Need to remove TTL for newly uploaded blobs too. Is that happening?



##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManagerUtil.java:
##########
@@ -411,4 +438,292 @@ public static Set<String> getSideInputStoreNames(
     }
     return sideInputStores;
   }
+
+  public static List<Future<Void>> initAndRestoreTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, SamzaContainerMetrics samzaContainerMetrics,
+      CheckpointManager checkpointManager, JobContext jobContext, ContainerModel containerModel,
+      Map<TaskName, Checkpoint> taskCheckpoints, Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames,
+      Config config, ExecutorService executor, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      File loggerStoreDir, Map<String, SystemConsumer> storeConsumers) {
+
+    Set<String> forceRestoreTasks = new HashSet<>();
+    // Initialize each TaskStorageManager.
+    taskRestoreManagers.forEach((taskName, restoreManagers) ->
+        restoreManagers.forEach((factoryName, taskRestoreManager) -> {
+          try {
+            taskRestoreManager.init(taskCheckpoints.get(taskName));
+          } catch (SamzaException ex) {
+            if (isUnwrappedExceptionDeletedException(ex) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+              // Get deleted SnapshotIndex blob with GetDeleted and mark the task to be restored with GetDeleted as well.
+              // this ensures that the restore downloads the snapshot, recreates a new snapshot, uploads it to blob store
+              // and creates a new checkpoint.
+              ((BlobStoreRestoreManager) taskRestoreManager).init(taskCheckpoints.get(taskName), true);
+              forceRestoreTasks.add(taskName.getTaskName());
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("init failed for task: %s with GetDeleted set to true", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex);
+            }
+          }
+        })
+    );
+
+    // Start each store consumer once.
+    // Note: These consumers are per system and only changelog system store consumers will be started.
+    // Some TaskRestoreManagers may not require the consumer to be started, but due to the agnostic nature of
+    // ContainerStorageManager we always start the changelog consumer here in case it is required
+    storeConsumers.values().stream().distinct().forEach(SystemConsumer::start);
+
+    return restoreAllTaskInstances(taskRestoreManagers, taskCheckpoints, taskBackendFactoryToStoreNames, jobContext,
+        containerModel, samzaContainerMetrics, checkpointManager, config, taskInstanceMetrics, executor, loggerStoreDir,
+        forceRestoreTasks);
+  }
+
+  /**
+   * Restores all TaskInstances and returns a future for each TaskInstance restore. If this restore fails with DeletedException
+   * it will retry the restore with getDeleted flag, get all the blobs, recreate a new checkpoint in blob store and
+   * write that checkpoint to checkpoint topic.
+   * @param taskRestoreManagers map of TaskName to factory name to TaskRestoreManager map.
+   */
+  public static List<Future<Void>> restoreAllTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, Map<TaskName, Checkpoint> taskCheckpoints,
+      Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames, JobContext jobContext,
+      ContainerModel containerModel, SamzaContainerMetrics samzaContainerMetrics, CheckpointManager checkpointManager,
+      Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics, ExecutorService executor,
+      File loggedStoreDir, Set<String> forceRestoreTask) {
+
+    List<Future<Void>> taskRestoreFutures = new ArrayList<>();
+
+    // Submit restore callable for each taskInstance
+    taskRestoreManagers.forEach((taskInstanceName, restoreManagersMap) -> {
+      // Submit for each restore factory
+      restoreManagersMap.forEach((factoryName, taskRestoreManager) -> {
+        long startTime = System.currentTimeMillis();
+        String taskName = taskInstanceName.getTaskName();
+        LOG.info("Starting restore for state for task: {}", taskName);
+
+        CompletableFuture<Void> restoreFuture;
+        if (forceRestoreTask.contains(taskName) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+          restoreFuture = ((BlobStoreRestoreManager) taskRestoreManager).restore(true);
+        } else {
+          restoreFuture = taskRestoreManager.restore();
+        }
+
+        CompletableFuture<Void> taskRestoreFuture = restoreFuture.handle((res, ex) -> {
+          updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+
+          if (ex != null) {
+            if (isUnwrappedExceptionDeletedException(ex)) {
+              LOG.warn(
+                  "Restore state for task: {} received DeletedException. Reattempting with getDeletedBlobs enabled",
+                  taskInstanceName.getTaskName());
+
+              // Try to restore with getDeleted flag
+              CompletableFuture<Void> future =
+                  restoreTaskStoresAndCreateCheckpointWithGetDeleted(taskInstanceName, taskCheckpoints, checkpointManager,
+                      taskRestoreManager, config, taskInstanceMetrics, executor,
+                      taskBackendFactoryToStoreNames.get(taskInstanceName).get(factoryName), loggedStoreDir, jobContext,
+                      containerModel);
+              try {
+                if (future != null) {
+                  future.join(); // Block and restore deleted state before continuing
+                }
+              } catch (Exception e) {
+                String msg = String.format("Unable to recover from DeletedException for task %s.", taskName);
+                throw new SamzaException(msg, e);
+              } finally {
+                updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+              }
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("Error restoring state for task: %s", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex); // wrap in unchecked exception to throw from lambda
+            }
+          }
+
+          // Stop all persistent stores after restoring. Certain persistent stores opened in BulkLoad mode are compacted
+          // on stop, so paralleling stop() also parallelizes their compaction (a time-intensive operation).
+          try {
+            taskRestoreManager.close();
+          } catch (Exception e) {
+            LOG.error("Error closing restore manager for task: {} after {} restore", taskName,
+                ex != null ? "unsuccessful" : "successful", e);
+            // ignore exception from close. container may still be able to continue processing/backups
+            // if restore manager close fails.
+          }
+          return null;
+        });
+        taskRestoreFutures.add(taskRestoreFuture);
+      });
+    });
+
+    return taskRestoreFutures;
+  }
+
+  public static CompletableFuture<Void> restoreTaskStoresAndCreateCheckpointWithGetDeleted(TaskName taskName,
+      Map<TaskName, Checkpoint> taskCheckpoints, CheckpointManager checkpointManager,
+      TaskRestoreManager taskRestoreManager, Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      ExecutorService executor, Set<String> storesToRestore, File loggedStoreBaseDirectory, JobContext jobContext,
+      ContainerModel containerModel) {
+
+    // if taskInstanceMetrics are specified use those for store metrics,
+    // otherwise (in case of StorageRecovery) use a blank MetricsRegistryMap
+    MetricsRegistry metricsRegistry =
+        taskInstanceMetrics.get(taskName) != null ? taskInstanceMetrics.get(taskName).registry()
+            : new MetricsRegistryMap();
+
+    BlobStoreManager blobStoreManager = getBlobStoreManager(config, executor);
+    JobConfig jobConfig = new JobConfig(config);
+    BlobStoreUtil blobStoreUtil =
+        new BlobStoreUtil(blobStoreManager, executor, new BlobStoreConfig(config), null,
+            new BlobStoreRestoreManagerMetrics(metricsRegistry));
+
+    BlobStoreRestoreManager blobStoreRestoreManager = (BlobStoreRestoreManager) taskRestoreManager;
+
+    CheckpointId checkpointId = CheckpointId.create();
+    Map<String, String> oldSCMs = ((CheckpointV2) taskCheckpoints.get(taskName))
+        .getStateCheckpointMarkers().get(BlobStoreStateBackendFactory.class.getName());
+
+    // Returns a single future that guarantees all the following are completed, in this order:
+    // 1. Restore state locally by getting deleted blobs from the blob store.
+    // 2. Create a new Checkpoint from restored state and back it up on the blob store.

Review Comment:
   If this is referring to rocksdb checkpoint, don't need that impl detail here. Or at least clarify it's RocksDB checkpoint.



##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManagerUtil.java:
##########
@@ -411,4 +438,292 @@ public static Set<String> getSideInputStoreNames(
     }
     return sideInputStores;
   }
+
+  public static List<Future<Void>> initAndRestoreTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, SamzaContainerMetrics samzaContainerMetrics,
+      CheckpointManager checkpointManager, JobContext jobContext, ContainerModel containerModel,
+      Map<TaskName, Checkpoint> taskCheckpoints, Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames,
+      Config config, ExecutorService executor, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      File loggerStoreDir, Map<String, SystemConsumer> storeConsumers) {
+
+    Set<String> forceRestoreTasks = new HashSet<>();
+    // Initialize each TaskStorageManager.
+    taskRestoreManagers.forEach((taskName, restoreManagers) ->
+        restoreManagers.forEach((factoryName, taskRestoreManager) -> {
+          try {
+            taskRestoreManager.init(taskCheckpoints.get(taskName));
+          } catch (SamzaException ex) {
+            if (isUnwrappedExceptionDeletedException(ex) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+              // Get deleted SnapshotIndex blob with GetDeleted and mark the task to be restored with GetDeleted as well.
+              // this ensures that the restore downloads the snapshot, recreates a new snapshot, uploads it to blob store
+              // and creates a new checkpoint.
+              ((BlobStoreRestoreManager) taskRestoreManager).init(taskCheckpoints.get(taskName), true);
+              forceRestoreTasks.add(taskName.getTaskName());
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("init failed for task: %s with GetDeleted set to true", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex);
+            }
+          }
+        })
+    );
+
+    // Start each store consumer once.
+    // Note: These consumers are per system and only changelog system store consumers will be started.
+    // Some TaskRestoreManagers may not require the consumer to be started, but due to the agnostic nature of
+    // ContainerStorageManager we always start the changelog consumer here in case it is required
+    storeConsumers.values().stream().distinct().forEach(SystemConsumer::start);
+
+    return restoreAllTaskInstances(taskRestoreManagers, taskCheckpoints, taskBackendFactoryToStoreNames, jobContext,
+        containerModel, samzaContainerMetrics, checkpointManager, config, taskInstanceMetrics, executor, loggerStoreDir,
+        forceRestoreTasks);
+  }
+
+  /**
+   * Restores all TaskInstances and returns a future for each TaskInstance restore. If this restore fails with DeletedException
+   * it will retry the restore with getDeleted flag, get all the blobs, recreate a new checkpoint in blob store and
+   * write that checkpoint to checkpoint topic.
+   * @param taskRestoreManagers map of TaskName to factory name to TaskRestoreManager map.
+   */
+  public static List<Future<Void>> restoreAllTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, Map<TaskName, Checkpoint> taskCheckpoints,
+      Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames, JobContext jobContext,
+      ContainerModel containerModel, SamzaContainerMetrics samzaContainerMetrics, CheckpointManager checkpointManager,
+      Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics, ExecutorService executor,
+      File loggedStoreDir, Set<String> forceRestoreTask) {
+
+    List<Future<Void>> taskRestoreFutures = new ArrayList<>();
+
+    // Submit restore callable for each taskInstance
+    taskRestoreManagers.forEach((taskInstanceName, restoreManagersMap) -> {
+      // Submit for each restore factory
+      restoreManagersMap.forEach((factoryName, taskRestoreManager) -> {
+        long startTime = System.currentTimeMillis();
+        String taskName = taskInstanceName.getTaskName();
+        LOG.info("Starting restore for state for task: {}", taskName);
+
+        CompletableFuture<Void> restoreFuture;
+        if (forceRestoreTask.contains(taskName) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+          restoreFuture = ((BlobStoreRestoreManager) taskRestoreManager).restore(true);
+        } else {
+          restoreFuture = taskRestoreManager.restore();
+        }
+
+        CompletableFuture<Void> taskRestoreFuture = restoreFuture.handle((res, ex) -> {
+          updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+
+          if (ex != null) {
+            if (isUnwrappedExceptionDeletedException(ex)) {
+              LOG.warn(
+                  "Restore state for task: {} received DeletedException. Reattempting with getDeletedBlobs enabled",
+                  taskInstanceName.getTaskName());
+
+              // Try to restore with getDeleted flag
+              CompletableFuture<Void> future =
+                  restoreTaskStoresAndCreateCheckpointWithGetDeleted(taskInstanceName, taskCheckpoints, checkpointManager,
+                      taskRestoreManager, config, taskInstanceMetrics, executor,
+                      taskBackendFactoryToStoreNames.get(taskInstanceName).get(factoryName), loggedStoreDir, jobContext,
+                      containerModel);
+              try {
+                if (future != null) {
+                  future.join(); // Block and restore deleted state before continuing
+                }
+              } catch (Exception e) {
+                String msg = String.format("Unable to recover from DeletedException for task %s.", taskName);
+                throw new SamzaException(msg, e);
+              } finally {
+                updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+              }
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("Error restoring state for task: %s", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex); // wrap in unchecked exception to throw from lambda
+            }
+          }
+
+          // Stop all persistent stores after restoring. Certain persistent stores opened in BulkLoad mode are compacted
+          // on stop, so paralleling stop() also parallelizes their compaction (a time-intensive operation).
+          try {
+            taskRestoreManager.close();
+          } catch (Exception e) {
+            LOG.error("Error closing restore manager for task: {} after {} restore", taskName,
+                ex != null ? "unsuccessful" : "successful", e);
+            // ignore exception from close. container may still be able to continue processing/backups
+            // if restore manager close fails.
+          }
+          return null;
+        });
+        taskRestoreFutures.add(taskRestoreFuture);
+      });
+    });
+
+    return taskRestoreFutures;
+  }
+
+  public static CompletableFuture<Void> restoreTaskStoresAndCreateCheckpointWithGetDeleted(TaskName taskName,
+      Map<TaskName, Checkpoint> taskCheckpoints, CheckpointManager checkpointManager,
+      TaskRestoreManager taskRestoreManager, Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      ExecutorService executor, Set<String> storesToRestore, File loggedStoreBaseDirectory, JobContext jobContext,
+      ContainerModel containerModel) {
+
+    // if taskInstanceMetrics are specified use those for store metrics,
+    // otherwise (in case of StorageRecovery) use a blank MetricsRegistryMap
+    MetricsRegistry metricsRegistry =
+        taskInstanceMetrics.get(taskName) != null ? taskInstanceMetrics.get(taskName).registry()
+            : new MetricsRegistryMap();
+
+    BlobStoreManager blobStoreManager = getBlobStoreManager(config, executor);
+    JobConfig jobConfig = new JobConfig(config);
+    BlobStoreUtil blobStoreUtil =
+        new BlobStoreUtil(blobStoreManager, executor, new BlobStoreConfig(config), null,
+            new BlobStoreRestoreManagerMetrics(metricsRegistry));
+
+    BlobStoreRestoreManager blobStoreRestoreManager = (BlobStoreRestoreManager) taskRestoreManager;
+
+    CheckpointId checkpointId = CheckpointId.create();
+    Map<String, String> oldSCMs = ((CheckpointV2) taskCheckpoints.get(taskName))
+        .getStateCheckpointMarkers().get(BlobStoreStateBackendFactory.class.getName());
+
+    // Returns a single future that guarantees all the following are completed, in this order:
+    // 1. Restore state locally by getting deleted blobs from the blob store.
+    // 2. Create a new Checkpoint from restored state and back it up on the blob store.
+    // 3. Clean up old/deleted Snapshot
+    // 4. Remove TTL from the new Snapshot on the blob store
+    // 5. Write the new checkpoint to checkpoint topic and update taskCheckpoints map
+
+    // 1. restore state with getDeleted flag set to true
+    return blobStoreRestoreManager.restore(true).thenCompose(r -> {
+      // 2. Create and backup new checkpoint on the blob store.
+      CompletableFuture<Map<String, String>> uploadSCMs =
+          createNewCheckpointAndBackupStores(jobContext, containerModel, config, taskName, storesToRestore, checkpointId,
+              loggedStoreBaseDirectory, blobStoreManager, metricsRegistry, executor);
+      CompletableFuture<Void> future = uploadSCMs.thenCompose(scms -> {
+        // 3. Delete prev SnapshotIndex including files/subdirs,
+        Map<String, CompletionStage<Void>> deleteOldSnapshotsFutures =
+            deletePrevSnapshots(blobStoreUtil, oldSCMs, taskName, jobConfig);
+        // 4. Mark new Snapshots to never expire
+        Map<String, CompletionStage<Void>> removeNewSnapshotsTTLFutures =
+            removeTTLNewSnapshots(blobStoreUtil, scms, taskName, jobConfig);
+        // Combined future of delete old snapshots and removeTTL of new snapshots
+        CompletableFuture<Void> removeTTLFuture = CompletableFuture.allOf(FutureUtil.mapToFuture(deleteOldSnapshotsFutures),
+            FutureUtil.mapToFuture(removeNewSnapshotsTTLFutures));
+        // 5. Update taskCheckpoints map with new checkpoint
+        CompletableFuture<Void> checkpointFuture = removeTTLFuture.thenRun(() -> {
+          Checkpoint newCheckpoint = writeNewCheckpoint(taskName, checkpointId, scms, checkpointManager);
+          taskCheckpoints.put(taskName, newCheckpoint);
+        });
+        return checkpointFuture;
+      });
+      return future;
+    }).exceptionally(ex -> {
+      String msg = String.format("Restore for task %s failed with get deleted bob", taskName);
+      throw new SamzaException(msg, ex);
+    });
+  }
+
+  private static CompletableFuture<Map<String, String>> createNewCheckpointAndBackupStores(JobContext jobContext,
+      ContainerModel containerModel, Config config, TaskName taskName, Set<String> storesToBackup,
+      CheckpointId newCheckpointId, File loggedStoreBaseDirectory, BlobStoreManager blobStoreManager,
+      MetricsRegistry metricsRegistry, ExecutorService executor) {
+
+    BlobStoreBackupManagerMetrics blobStoreBackupManagerMetrics = new BlobStoreBackupManagerMetrics(metricsRegistry);
+    BlobStoreBackupManager blobStoreBackupManager =
+        new BlobStoreBackupManager(jobContext.getJobModel(), containerModel, containerModel.getTasks().get(taskName),
+            executor, blobStoreBackupManagerMetrics, config, SystemClock.instance(), loggedStoreBaseDirectory,
+            new StorageManagerUtil(), blobStoreManager);
+
+    // create checkpoint dir as a copy of store dir
+    createCheckpointDirFromStoreDirCopy(taskName, containerModel.getTasks().get(taskName),
+        loggedStoreBaseDirectory, storesToBackup, newCheckpointId);
+    // upload to blob store and return future
+    return blobStoreBackupManager.upload(newCheckpointId, new HashMap<>());
+  }
+
+  private static Checkpoint writeNewCheckpoint(TaskName taskName, CheckpointId checkpointId,
+      Map<String, String> newSCMs, CheckpointManager checkpointManager) {
+    CheckpointV2 oldCheckpoint = (CheckpointV2) checkpointManager.readLastCheckpoint(taskName);
+    Map<SystemStreamPartition, String> checkpointOffsets = oldCheckpoint.getOffsets();
+
+    Map<String, Map<String, String>> oldStateCheckpointMarkers = oldCheckpoint.getStateCheckpointMarkers();
+
+    Map<String, Map<String, String>> newStateCheckpointMarkers = ImmutableMap.<String, Map<String, String>>builder()
+        .put(KafkaChangelogStateBackendFactory.class.getName(), oldStateCheckpointMarkers.get(KafkaChangelogStateBackendFactory.class.getName()))

Review Comment:
   What if value isn't present (changelog is disabled)? Should this be adding null checkpoints?



##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManagerUtil.java:
##########
@@ -411,4 +438,292 @@ public static Set<String> getSideInputStoreNames(
     }
     return sideInputStores;
   }
+
+  public static List<Future<Void>> initAndRestoreTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, SamzaContainerMetrics samzaContainerMetrics,
+      CheckpointManager checkpointManager, JobContext jobContext, ContainerModel containerModel,
+      Map<TaskName, Checkpoint> taskCheckpoints, Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames,
+      Config config, ExecutorService executor, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      File loggerStoreDir, Map<String, SystemConsumer> storeConsumers) {
+
+    Set<String> forceRestoreTasks = new HashSet<>();
+    // Initialize each TaskStorageManager.
+    taskRestoreManagers.forEach((taskName, restoreManagers) ->
+        restoreManagers.forEach((factoryName, taskRestoreManager) -> {
+          try {
+            taskRestoreManager.init(taskCheckpoints.get(taskName));
+          } catch (SamzaException ex) {
+            if (isUnwrappedExceptionDeletedException(ex) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+              // Get deleted SnapshotIndex blob with GetDeleted and mark the task to be restored with GetDeleted as well.
+              // this ensures that the restore downloads the snapshot, recreates a new snapshot, uploads it to blob store
+              // and creates a new checkpoint.
+              ((BlobStoreRestoreManager) taskRestoreManager).init(taskCheckpoints.get(taskName), true);
+              forceRestoreTasks.add(taskName.getTaskName());
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("init failed for task: %s with GetDeleted set to true", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex);
+            }
+          }
+        })
+    );
+
+    // Start each store consumer once.
+    // Note: These consumers are per system and only changelog system store consumers will be started.
+    // Some TaskRestoreManagers may not require the consumer to be started, but due to the agnostic nature of
+    // ContainerStorageManager we always start the changelog consumer here in case it is required
+    storeConsumers.values().stream().distinct().forEach(SystemConsumer::start);
+
+    return restoreAllTaskInstances(taskRestoreManagers, taskCheckpoints, taskBackendFactoryToStoreNames, jobContext,
+        containerModel, samzaContainerMetrics, checkpointManager, config, taskInstanceMetrics, executor, loggerStoreDir,
+        forceRestoreTasks);
+  }
+
+  /**
+   * Restores all TaskInstances and returns a future for each TaskInstance restore. If this restore fails with DeletedException
+   * it will retry the restore with getDeleted flag, get all the blobs, recreate a new checkpoint in blob store and
+   * write that checkpoint to checkpoint topic.
+   * @param taskRestoreManagers map of TaskName to factory name to TaskRestoreManager map.
+   */
+  public static List<Future<Void>> restoreAllTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, Map<TaskName, Checkpoint> taskCheckpoints,
+      Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames, JobContext jobContext,
+      ContainerModel containerModel, SamzaContainerMetrics samzaContainerMetrics, CheckpointManager checkpointManager,
+      Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics, ExecutorService executor,
+      File loggedStoreDir, Set<String> forceRestoreTask) {
+
+    List<Future<Void>> taskRestoreFutures = new ArrayList<>();
+
+    // Submit restore callable for each taskInstance
+    taskRestoreManagers.forEach((taskInstanceName, restoreManagersMap) -> {
+      // Submit for each restore factory
+      restoreManagersMap.forEach((factoryName, taskRestoreManager) -> {
+        long startTime = System.currentTimeMillis();
+        String taskName = taskInstanceName.getTaskName();
+        LOG.info("Starting restore for state for task: {}", taskName);
+
+        CompletableFuture<Void> restoreFuture;
+        if (forceRestoreTask.contains(taskName) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+          restoreFuture = ((BlobStoreRestoreManager) taskRestoreManager).restore(true);
+        } else {
+          restoreFuture = taskRestoreManager.restore();
+        }
+
+        CompletableFuture<Void> taskRestoreFuture = restoreFuture.handle((res, ex) -> {
+          updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+
+          if (ex != null) {
+            if (isUnwrappedExceptionDeletedException(ex)) {
+              LOG.warn(
+                  "Restore state for task: {} received DeletedException. Reattempting with getDeletedBlobs enabled",
+                  taskInstanceName.getTaskName());
+
+              // Try to restore with getDeleted flag
+              CompletableFuture<Void> future =
+                  restoreTaskStoresAndCreateCheckpointWithGetDeleted(taskInstanceName, taskCheckpoints, checkpointManager,
+                      taskRestoreManager, config, taskInstanceMetrics, executor,
+                      taskBackendFactoryToStoreNames.get(taskInstanceName).get(factoryName), loggedStoreDir, jobContext,
+                      containerModel);
+              try {
+                if (future != null) {
+                  future.join(); // Block and restore deleted state before continuing
+                }
+              } catch (Exception e) {
+                String msg = String.format("Unable to recover from DeletedException for task %s.", taskName);
+                throw new SamzaException(msg, e);
+              } finally {
+                updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+              }
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("Error restoring state for task: %s", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex); // wrap in unchecked exception to throw from lambda
+            }
+          }
+
+          // Stop all persistent stores after restoring. Certain persistent stores opened in BulkLoad mode are compacted
+          // on stop, so paralleling stop() also parallelizes their compaction (a time-intensive operation).
+          try {
+            taskRestoreManager.close();
+          } catch (Exception e) {
+            LOG.error("Error closing restore manager for task: {} after {} restore", taskName,
+                ex != null ? "unsuccessful" : "successful", e);
+            // ignore exception from close. container may still be able to continue processing/backups
+            // if restore manager close fails.
+          }
+          return null;
+        });
+        taskRestoreFutures.add(taskRestoreFuture);
+      });
+    });
+
+    return taskRestoreFutures;
+  }
+
+  public static CompletableFuture<Void> restoreTaskStoresAndCreateCheckpointWithGetDeleted(TaskName taskName,
+      Map<TaskName, Checkpoint> taskCheckpoints, CheckpointManager checkpointManager,
+      TaskRestoreManager taskRestoreManager, Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      ExecutorService executor, Set<String> storesToRestore, File loggedStoreBaseDirectory, JobContext jobContext,
+      ContainerModel containerModel) {
+
+    // if taskInstanceMetrics are specified use those for store metrics,
+    // otherwise (in case of StorageRecovery) use a blank MetricsRegistryMap
+    MetricsRegistry metricsRegistry =
+        taskInstanceMetrics.get(taskName) != null ? taskInstanceMetrics.get(taskName).registry()
+            : new MetricsRegistryMap();
+
+    BlobStoreManager blobStoreManager = getBlobStoreManager(config, executor);
+    JobConfig jobConfig = new JobConfig(config);
+    BlobStoreUtil blobStoreUtil =
+        new BlobStoreUtil(blobStoreManager, executor, new BlobStoreConfig(config), null,
+            new BlobStoreRestoreManagerMetrics(metricsRegistry));
+
+    BlobStoreRestoreManager blobStoreRestoreManager = (BlobStoreRestoreManager) taskRestoreManager;
+
+    CheckpointId checkpointId = CheckpointId.create();
+    Map<String, String> oldSCMs = ((CheckpointV2) taskCheckpoints.get(taskName))
+        .getStateCheckpointMarkers().get(BlobStoreStateBackendFactory.class.getName());
+
+    // Returns a single future that guarantees all the following are completed, in this order:
+    // 1. Restore state locally by getting deleted blobs from the blob store.
+    // 2. Create a new Checkpoint from restored state and back it up on the blob store.
+    // 3. Clean up old/deleted Snapshot
+    // 4. Remove TTL from the new Snapshot on the blob store
+    // 5. Write the new checkpoint to checkpoint topic and update taskCheckpoints map
+
+    // 1. restore state with getDeleted flag set to true
+    return blobStoreRestoreManager.restore(true).thenCompose(r -> {
+      // 2. Create and backup new checkpoint on the blob store.
+      CompletableFuture<Map<String, String>> uploadSCMs =
+          createNewCheckpointAndBackupStores(jobContext, containerModel, config, taskName, storesToRestore, checkpointId,
+              loggedStoreBaseDirectory, blobStoreManager, metricsRegistry, executor);
+      CompletableFuture<Void> future = uploadSCMs.thenCompose(scms -> {
+        // 3. Delete prev SnapshotIndex including files/subdirs,
+        Map<String, CompletionStage<Void>> deleteOldSnapshotsFutures =
+            deletePrevSnapshots(blobStoreUtil, oldSCMs, taskName, jobConfig);
+        // 4. Mark new Snapshots to never expire
+        Map<String, CompletionStage<Void>> removeNewSnapshotsTTLFutures =
+            removeTTLNewSnapshots(blobStoreUtil, scms, taskName, jobConfig);
+        // Combined future of delete old snapshots and removeTTL of new snapshots
+        CompletableFuture<Void> removeTTLFuture = CompletableFuture.allOf(FutureUtil.mapToFuture(deleteOldSnapshotsFutures),
+            FutureUtil.mapToFuture(removeNewSnapshotsTTLFutures));
+        // 5. Update taskCheckpoints map with new checkpoint
+        CompletableFuture<Void> checkpointFuture = removeTTLFuture.thenRun(() -> {
+          Checkpoint newCheckpoint = writeNewCheckpoint(taskName, checkpointId, scms, checkpointManager);
+          taskCheckpoints.put(taskName, newCheckpoint);
+        });
+        return checkpointFuture;
+      });
+      return future;
+    }).exceptionally(ex -> {
+      String msg = String.format("Restore for task %s failed with get deleted bob", taskName);
+      throw new SamzaException(msg, ex);
+    });
+  }
+
+  private static CompletableFuture<Map<String, String>> createNewCheckpointAndBackupStores(JobContext jobContext,
+      ContainerModel containerModel, Config config, TaskName taskName, Set<String> storesToBackup,
+      CheckpointId newCheckpointId, File loggedStoreBaseDirectory, BlobStoreManager blobStoreManager,
+      MetricsRegistry metricsRegistry, ExecutorService executor) {
+
+    BlobStoreBackupManagerMetrics blobStoreBackupManagerMetrics = new BlobStoreBackupManagerMetrics(metricsRegistry);
+    BlobStoreBackupManager blobStoreBackupManager =
+        new BlobStoreBackupManager(jobContext.getJobModel(), containerModel, containerModel.getTasks().get(taskName),
+            executor, blobStoreBackupManagerMetrics, config, SystemClock.instance(), loggedStoreBaseDirectory,
+            new StorageManagerUtil(), blobStoreManager);
+
+    // create checkpoint dir as a copy of store dir
+    createCheckpointDirFromStoreDirCopy(taskName, containerModel.getTasks().get(taskName),
+        loggedStoreBaseDirectory, storesToBackup, newCheckpointId);
+    // upload to blob store and return future
+    return blobStoreBackupManager.upload(newCheckpointId, new HashMap<>());
+  }
+
+  private static Checkpoint writeNewCheckpoint(TaskName taskName, CheckpointId checkpointId,
+      Map<String, String> newSCMs, CheckpointManager checkpointManager) {
+    CheckpointV2 oldCheckpoint = (CheckpointV2) checkpointManager.readLastCheckpoint(taskName);
+    Map<SystemStreamPartition, String> checkpointOffsets = oldCheckpoint.getOffsets();
+
+    Map<String, Map<String, String>> oldStateCheckpointMarkers = oldCheckpoint.getStateCheckpointMarkers();
+
+    Map<String, Map<String, String>> newStateCheckpointMarkers = ImmutableMap.<String, Map<String, String>>builder()
+        .put(KafkaChangelogStateBackendFactory.class.getName(), oldStateCheckpointMarkers.get(KafkaChangelogStateBackendFactory.class.getName()))
+        .put(BlobStoreStateBackendFactory.class.getName(), newSCMs)
+        .build();
+
+    CheckpointV2 checkpointV2 = new CheckpointV2(checkpointId, checkpointOffsets, newStateCheckpointMarkers);
+    checkpointManager.writeCheckpoint(taskName, checkpointV2);
+    return checkpointV2;
+  }
+
+  private static void createCheckpointDirFromStoreDirCopy(TaskName taskName, TaskModel taskModel,

Review Comment:
   Let's move all upload/download/restore related util methods to a different util class.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] shekhars-li commented on a diff in pull request #1676: [SAMZA-2787] GetDeleted API and Recover from DeletedException

Posted by "shekhars-li (via GitHub)" <gi...@apache.org>.
shekhars-li commented on code in PR #1676:
URL: https://github.com/apache/samza/pull/1676#discussion_r1275563417


##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManagerUtil.java:
##########
@@ -411,4 +438,292 @@ public static Set<String> getSideInputStoreNames(
     }
     return sideInputStores;
   }
+
+  public static List<Future<Void>> initAndRestoreTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, SamzaContainerMetrics samzaContainerMetrics,
+      CheckpointManager checkpointManager, JobContext jobContext, ContainerModel containerModel,
+      Map<TaskName, Checkpoint> taskCheckpoints, Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames,
+      Config config, ExecutorService executor, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      File loggerStoreDir, Map<String, SystemConsumer> storeConsumers) {
+
+    Set<String> forceRestoreTasks = new HashSet<>();
+    // Initialize each TaskStorageManager.
+    taskRestoreManagers.forEach((taskName, restoreManagers) ->
+        restoreManagers.forEach((factoryName, taskRestoreManager) -> {
+          try {
+            taskRestoreManager.init(taskCheckpoints.get(taskName));
+          } catch (SamzaException ex) {
+            if (isUnwrappedExceptionDeletedException(ex) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+              // Get deleted SnapshotIndex blob with GetDeleted and mark the task to be restored with GetDeleted as well.
+              // this ensures that the restore downloads the snapshot, recreates a new snapshot, uploads it to blob store
+              // and creates a new checkpoint.
+              ((BlobStoreRestoreManager) taskRestoreManager).init(taskCheckpoints.get(taskName), true);
+              forceRestoreTasks.add(taskName.getTaskName());
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("init failed for task: %s with GetDeleted set to true", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex);
+            }
+          }
+        })
+    );
+
+    // Start each store consumer once.
+    // Note: These consumers are per system and only changelog system store consumers will be started.
+    // Some TaskRestoreManagers may not require the consumer to be started, but due to the agnostic nature of
+    // ContainerStorageManager we always start the changelog consumer here in case it is required
+    storeConsumers.values().stream().distinct().forEach(SystemConsumer::start);
+
+    return restoreAllTaskInstances(taskRestoreManagers, taskCheckpoints, taskBackendFactoryToStoreNames, jobContext,
+        containerModel, samzaContainerMetrics, checkpointManager, config, taskInstanceMetrics, executor, loggerStoreDir,
+        forceRestoreTasks);
+  }
+
+  /**
+   * Restores all TaskInstances and returns a future for each TaskInstance restore. If this restore fails with DeletedException
+   * it will retry the restore with getDeleted flag, get all the blobs, recreate a new checkpoint in blob store and
+   * write that checkpoint to checkpoint topic.
+   * @param taskRestoreManagers map of TaskName to factory name to TaskRestoreManager map.
+   */
+  public static List<Future<Void>> restoreAllTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, Map<TaskName, Checkpoint> taskCheckpoints,
+      Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames, JobContext jobContext,
+      ContainerModel containerModel, SamzaContainerMetrics samzaContainerMetrics, CheckpointManager checkpointManager,
+      Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics, ExecutorService executor,
+      File loggedStoreDir, Set<String> forceRestoreTask) {
+
+    List<Future<Void>> taskRestoreFutures = new ArrayList<>();
+
+    // Submit restore callable for each taskInstance
+    taskRestoreManagers.forEach((taskInstanceName, restoreManagersMap) -> {
+      // Submit for each restore factory
+      restoreManagersMap.forEach((factoryName, taskRestoreManager) -> {
+        long startTime = System.currentTimeMillis();
+        String taskName = taskInstanceName.getTaskName();
+        LOG.info("Starting restore for state for task: {}", taskName);
+
+        CompletableFuture<Void> restoreFuture;
+        if (forceRestoreTask.contains(taskName) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+          restoreFuture = ((BlobStoreRestoreManager) taskRestoreManager).restore(true);
+        } else {
+          restoreFuture = taskRestoreManager.restore();
+        }
+
+        CompletableFuture<Void> taskRestoreFuture = restoreFuture.handle((res, ex) -> {
+          updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+
+          if (ex != null) {
+            if (isUnwrappedExceptionDeletedException(ex)) {
+              LOG.warn(
+                  "Restore state for task: {} received DeletedException. Reattempting with getDeletedBlobs enabled",
+                  taskInstanceName.getTaskName());
+
+              // Try to restore with getDeleted flag
+              CompletableFuture<Void> future =
+                  restoreTaskStoresAndCreateCheckpointWithGetDeleted(taskInstanceName, taskCheckpoints, checkpointManager,
+                      taskRestoreManager, config, taskInstanceMetrics, executor,
+                      taskBackendFactoryToStoreNames.get(taskInstanceName).get(factoryName), loggedStoreDir, jobContext,
+                      containerModel);
+              try {
+                if (future != null) {
+                  future.join(); // Block and restore deleted state before continuing

Review Comment:
   Yes, restore is happening on restore executor. So is all the recovery related executions (upload/ttl removal etc)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] shekhars-li commented on a diff in pull request #1676: [SAMZA-2787] GetDeleted API and Recover from DeletedException

Posted by "shekhars-li (via GitHub)" <gi...@apache.org>.
shekhars-li commented on code in PR #1676:
URL: https://github.com/apache/samza/pull/1676#discussion_r1286261917


##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreRestoreManager.java:
##########
@@ -119,13 +119,18 @@ public BlobStoreRestoreManager(TaskModel taskModel, ExecutorService restoreExecu
 
   @Override
   public void init(Checkpoint checkpoint) {
+    init(checkpoint, false);
+  }
+
+  public void init(Checkpoint checkpoint, Boolean getDeletedBlob) {

Review Comment:
   This is needed because TaskRestoreManager interface has init(Checkpoint). We only need getDeleted version of it for blob store, so created another version here.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] shekhars-li commented on a diff in pull request #1676: [SAMZA-2787] GetDeleted API and Recover from DeletedException

Posted by "shekhars-li (via GitHub)" <gi...@apache.org>.
shekhars-li commented on code in PR #1676:
URL: https://github.com/apache/samza/pull/1676#discussion_r1286275959


##########
samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala:
##########
@@ -149,31 +149,33 @@ class TaskInstance(
     }
   }
 
-  def initTask {
+  def initTask(lastTaskCheckpoint: Checkpoint) {
     initCaughtUpMapping()
 
+    val isStandByTask = taskModel.getTaskMode == TaskMode.Standby
+
     if (commitManager != null) {
       debug("Starting commit manager for taskName: %s" format taskName)
-
-      commitManager.init()
+      commitManager.init(if (isStandByTask) null else lastTaskCheckpoint)
     } else {
       debug("Skipping commit manager initialization for taskName: %s" format taskName)
     }
 
-    if (offsetManager != null) {
-      val checkpoint = offsetManager.getLastTaskCheckpoint(taskName)
-      // Only required for checkpointV2
-      if (checkpoint != null && checkpoint.getVersion == 2) {
-        val checkpointV2 = checkpoint.asInstanceOf[CheckpointV2]
-        // call cleanUp on backup managers in case the container previously failed during commit
-        // before completing this step
-
-        // WARNING: cleanUp is NOT optional with blob stores since this is where we reset the TTL for
-        // tracked blobs. if this TTL reset is skipped, some of the blobs retained by future commits may
-        // be deleted in the background by the blob store, leading to data loss.
-        info("Cleaning up stale state from previous run for taskName: %s" format taskName)
-        commitManager.cleanUp(checkpointV2.getCheckpointId, checkpointV2.getStateCheckpointMarkers)
-      }
+    var checkpoint: Checkpoint = lastTaskCheckpoint
+    if (offsetManager != null && isStandByTask) {

Review Comment:
   Added a comment. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] dxichen merged pull request #1676: [SAMZA-2787] GetDeleted API and Recover from DeletedException

Posted by "dxichen (via GitHub)" <gi...@apache.org>.
dxichen merged PR #1676:
URL: https://github.com/apache/samza/pull/1676


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] shekhars-li commented on a diff in pull request #1676: [SAMZA-2787] GetDeleted API and Recover from DeletedException

Posted by "shekhars-li (via GitHub)" <gi...@apache.org>.
shekhars-li commented on code in PR #1676:
URL: https://github.com/apache/samza/pull/1676#discussion_r1275518893


##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java:
##########
@@ -400,6 +349,7 @@ private void restoreStores() throws InterruptedException {
     });
 
     LOG.info("Store Restore complete");
+    return taskCheckpoints;

Review Comment:
   The new checkpoints are reflected in the taskCheckpoints.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] shekhars-li commented on a diff in pull request #1676: [SAMZA-2787] GetDeleted API and Recover from DeletedException

Posted by "shekhars-li (via GitHub)" <gi...@apache.org>.
shekhars-li commented on code in PR #1676:
URL: https://github.com/apache/samza/pull/1676#discussion_r1286275278


##########
samza-core/src/main/java/org/apache/samza/util/FutureUtil.java:
##########
@@ -139,6 +139,17 @@ public static <K, V> CompletableFuture<Map<K, V>> toFutureOfMap(
     });
   }
 
+  /**
+   * Helper method to convert: {@code Map<K, CompletionStage<Void>>}
+   * to:                       {@code CompletableFuture<Void>}
+   *
+   * Returns a future that completes when all value futures complete.
+   * Returned future completes exceptionally if any of the value futures complete exceptionally.
+   */
+  public static CompletableFuture<Void> mapToFuture(Map<String, CompletionStage<Void>> map) {

Review Comment:
   They are different methods. This one creates a future to ensure all removeTTLFutures in the map completes. We don't care about individual k, v. The other method returns a future of map. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] shekhars-li commented on a diff in pull request #1676: [SAMZA-2787] GetDeleted API and Recover from DeletedException

Posted by "shekhars-li (via GitHub)" <gi...@apache.org>.
shekhars-li commented on code in PR #1676:
URL: https://github.com/apache/samza/pull/1676#discussion_r1286275788


##########
samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala:
##########
@@ -149,31 +149,33 @@ class TaskInstance(
     }
   }
 
-  def initTask {
+  def initTask(lastTaskCheckpoint: Checkpoint) {
     initCaughtUpMapping()
 
+    val isStandByTask = taskModel.getTaskMode == TaskMode.Standby
+
     if (commitManager != null) {
       debug("Starting commit manager for taskName: %s" format taskName)
-
-      commitManager.init()
+      commitManager.init(if (isStandByTask) null else lastTaskCheckpoint)

Review Comment:
   Updated, thanks!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] shekhars-li commented on a diff in pull request #1676: [SAMZA-2787] GetDeleted API and Recover from DeletedException

Posted by "shekhars-li (via GitHub)" <gi...@apache.org>.
shekhars-li commented on code in PR #1676:
URL: https://github.com/apache/samza/pull/1676#discussion_r1287910302


##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreRestoreManager.java:
##########
@@ -254,8 +263,10 @@ static CompletableFuture<Void> restoreStores(String jobName, String jobId, TaskN
         throw new SamzaException(String.format("Error deleting store directory: %s", storeDir), e);
       }
 
+      // If getDeletedBlobs is enabled - always restore so that we get all the blobs, including the deleted blobs,
+      // immediately restore it locally and backup to create new checkpoint.

Review Comment:
   Thanks, clarified. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] shekhars-li commented on a diff in pull request #1676: [SAMZA-2787] GetDeleted API and Recover from DeletedException

Posted by "shekhars-li (via GitHub)" <gi...@apache.org>.
shekhars-li commented on code in PR #1676:
URL: https://github.com/apache/samza/pull/1676#discussion_r1289216311


##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManagerUtil.java:
##########
@@ -411,4 +438,292 @@ public static Set<String> getSideInputStoreNames(
     }
     return sideInputStores;
   }
+
+  /**
+   * Inits and Restores all the task stores.
+   * Note: In case of {@link BlobStoreRestoreManager}, this method retries init and restore with getDeleted flag if it
+   * receives a {@link DeletedException}. This will create a new checkpoint for the corresponding task.
+   */
+  public static CompletableFuture<Map<TaskName, Checkpoint>> initAndRestoreTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, SamzaContainerMetrics samzaContainerMetrics,
+      CheckpointManager checkpointManager, JobContext jobContext, ContainerModel containerModel,
+      Map<TaskName, Checkpoint> taskCheckpoints, Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames,
+      Config config, ExecutorService executor, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      File loggerStoreDir, Map<String, SystemConsumer> storeConsumers) {
+
+    Set<String> forceRestoreTasks = new HashSet<>();
+    // Initialize each TaskStorageManager.
+    taskRestoreManagers.forEach((taskName, restoreManagers) ->
+        restoreManagers.forEach((factoryName, taskRestoreManager) -> {
+          try {
+            taskRestoreManager.init(taskCheckpoints.get(taskName));
+          } catch (SamzaException ex) {
+            if (isUnwrappedExceptionDeletedException(ex) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+              // Get deleted SnapshotIndex blob with GetDeleted and mark the task to be restored with GetDeleted as well.
+              // this ensures that the restore downloads the snapshot, recreates a new snapshot, uploads it to blob store
+              // and creates a new checkpoint.
+              ((BlobStoreRestoreManager) taskRestoreManager).init(taskCheckpoints.get(taskName), true);
+              forceRestoreTasks.add(taskName.getTaskName());
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("init failed for BlobStoreRestoreManager, task: %s with GetDeleted set to true", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex);
+            }
+          }
+        })
+    );
+
+    // Start each store consumer once.
+    // Note: These consumers are per system and only changelog system store consumers will be started.
+    // Some TaskRestoreManagers may not require the consumer to be started, but due to the agnostic nature of
+    // ContainerStorageManager we always start the changelog consumer here in case it is required
+    storeConsumers.values().stream().distinct().forEach(SystemConsumer::start);
+
+    return restoreAllTaskInstances(taskRestoreManagers, taskCheckpoints, taskBackendFactoryToStoreNames, jobContext,
+        containerModel, samzaContainerMetrics, checkpointManager, config, taskInstanceMetrics, executor, loggerStoreDir,
+        forceRestoreTasks);
+  }
+
+  /**
+   * Restores all TaskInstances and returns a future for each TaskInstance restore. Note: In case of
+   * {@link BlobStoreRestoreManager}, this method restore with getDeleted flag if it receives a
+   * {@link DeletedException}. This will create a new Checkpoint.
+   */
+  private static CompletableFuture<Map<TaskName, Checkpoint>> restoreAllTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, Map<TaskName, Checkpoint> taskCheckpoints,
+      Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames, JobContext jobContext,
+      ContainerModel containerModel, SamzaContainerMetrics samzaContainerMetrics, CheckpointManager checkpointManager,
+      Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics, ExecutorService executor,
+      File loggedStoreDir, Set<String> forceRestoreTask) {
+
+    Map<TaskName, CompletableFuture<Checkpoint>> newTaskCheckpoints = new ConcurrentHashMap<>();
+    List<Future<Void>> taskRestoreFutures = new ArrayList<>();
+
+    // Submit restore callable for each taskInstance
+    taskRestoreManagers.forEach((taskInstanceName, restoreManagersMap) -> {
+      // Submit for each restore factory
+      restoreManagersMap.forEach((factoryName, taskRestoreManager) -> {
+        long startTime = System.currentTimeMillis();
+        String taskName = taskInstanceName.getTaskName();
+        LOG.info("Starting restore for state for task: {}", taskName);
+
+        CompletableFuture<Void> restoreFuture;
+        if (forceRestoreTask.contains(taskName) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+          // If init was retried with getDeleted, force restore with getDeleted as well, since init only inits the
+          // restoreManager with deleted SnapshotIndex but does not retry to recover the deleted blobs and delegates it
+          // to restore().
+          // Create an empty future that fails immediately with DeletedException to force retry in restore.
+          restoreFuture = new CompletableFuture<>();
+          restoreFuture.completeExceptionally(new SamzaException(new DeletedException()));
+        } else {
+          restoreFuture = taskRestoreManager.restore();
+        }
+
+        CompletableFuture<Void> taskRestoreFuture = restoreFuture.handle((res, ex) -> {
+          updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+
+          if (ex != null) {
+            if (isUnwrappedExceptionDeletedException(ex)) {
+              LOG.warn(
+                  "Received DeletedException during restore for task {}. Reattempting to get deleted blobs",
+                  taskInstanceName.getTaskName());
+
+              // Try to restore with getDeleted flag
+              CompletableFuture<Checkpoint> future =
+                  restoreDeletedSnapshot(taskInstanceName, taskCheckpoints,
+                      checkpointManager, taskRestoreManager, config, taskInstanceMetrics, executor,
+                      taskBackendFactoryToStoreNames.get(taskInstanceName).get(factoryName), loggedStoreDir,
+                      jobContext, containerModel);
+              try {
+                if (future != null) {
+                  newTaskCheckpoints.put(taskInstanceName, future);
+                }
+              } catch (Exception e) {

Review Comment:
   The method restoreDeletedSnapshot returns future with exceptionally chained to it and should catch any synchronous exceptions (like in writeNewCheckpoint). 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] shekhars-li commented on a diff in pull request #1676: [SAMZA-2787] GetDeleted API and Recover from DeletedException

Posted by "shekhars-li (via GitHub)" <gi...@apache.org>.
shekhars-li commented on code in PR #1676:
URL: https://github.com/apache/samza/pull/1676#discussion_r1286282038


##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManagerUtil.java:
##########
@@ -411,4 +438,292 @@ public static Set<String> getSideInputStoreNames(
     }
     return sideInputStores;
   }
+
+  public static List<Future<Void>> initAndRestoreTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, SamzaContainerMetrics samzaContainerMetrics,
+      CheckpointManager checkpointManager, JobContext jobContext, ContainerModel containerModel,
+      Map<TaskName, Checkpoint> taskCheckpoints, Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames,
+      Config config, ExecutorService executor, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      File loggerStoreDir, Map<String, SystemConsumer> storeConsumers) {
+
+    Set<String> forceRestoreTasks = new HashSet<>();
+    // Initialize each TaskStorageManager.
+    taskRestoreManagers.forEach((taskName, restoreManagers) ->
+        restoreManagers.forEach((factoryName, taskRestoreManager) -> {
+          try {
+            taskRestoreManager.init(taskCheckpoints.get(taskName));
+          } catch (SamzaException ex) {
+            if (isUnwrappedExceptionDeletedException(ex) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+              // Get deleted SnapshotIndex blob with GetDeleted and mark the task to be restored with GetDeleted as well.
+              // this ensures that the restore downloads the snapshot, recreates a new snapshot, uploads it to blob store
+              // and creates a new checkpoint.
+              ((BlobStoreRestoreManager) taskRestoreManager).init(taskCheckpoints.get(taskName), true);
+              forceRestoreTasks.add(taskName.getTaskName());
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("init failed for task: %s with GetDeleted set to true", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex);
+            }
+          }
+        })
+    );
+
+    // Start each store consumer once.
+    // Note: These consumers are per system and only changelog system store consumers will be started.
+    // Some TaskRestoreManagers may not require the consumer to be started, but due to the agnostic nature of
+    // ContainerStorageManager we always start the changelog consumer here in case it is required
+    storeConsumers.values().stream().distinct().forEach(SystemConsumer::start);
+
+    return restoreAllTaskInstances(taskRestoreManagers, taskCheckpoints, taskBackendFactoryToStoreNames, jobContext,
+        containerModel, samzaContainerMetrics, checkpointManager, config, taskInstanceMetrics, executor, loggerStoreDir,
+        forceRestoreTasks);
+  }
+
+  /**
+   * Restores all TaskInstances and returns a future for each TaskInstance restore. If this restore fails with DeletedException
+   * it will retry the restore with getDeleted flag, get all the blobs, recreate a new checkpoint in blob store and
+   * write that checkpoint to checkpoint topic.
+   * @param taskRestoreManagers map of TaskName to factory name to TaskRestoreManager map.
+   */
+  public static List<Future<Void>> restoreAllTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, Map<TaskName, Checkpoint> taskCheckpoints,
+      Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames, JobContext jobContext,
+      ContainerModel containerModel, SamzaContainerMetrics samzaContainerMetrics, CheckpointManager checkpointManager,
+      Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics, ExecutorService executor,
+      File loggedStoreDir, Set<String> forceRestoreTask) {
+
+    List<Future<Void>> taskRestoreFutures = new ArrayList<>();
+
+    // Submit restore callable for each taskInstance
+    taskRestoreManagers.forEach((taskInstanceName, restoreManagersMap) -> {
+      // Submit for each restore factory
+      restoreManagersMap.forEach((factoryName, taskRestoreManager) -> {
+        long startTime = System.currentTimeMillis();
+        String taskName = taskInstanceName.getTaskName();
+        LOG.info("Starting restore for state for task: {}", taskName);
+
+        CompletableFuture<Void> restoreFuture;
+        if (forceRestoreTask.contains(taskName) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+          restoreFuture = ((BlobStoreRestoreManager) taskRestoreManager).restore(true);
+        } else {
+          restoreFuture = taskRestoreManager.restore();
+        }
+
+        CompletableFuture<Void> taskRestoreFuture = restoreFuture.handle((res, ex) -> {
+          updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+
+          if (ex != null) {
+            if (isUnwrappedExceptionDeletedException(ex)) {
+              LOG.warn(
+                  "Restore state for task: {} received DeletedException. Reattempting with getDeletedBlobs enabled",
+                  taskInstanceName.getTaskName());
+
+              // Try to restore with getDeleted flag
+              CompletableFuture<Void> future =
+                  restoreTaskStoresAndCreateCheckpointWithGetDeleted(taskInstanceName, taskCheckpoints, checkpointManager,
+                      taskRestoreManager, config, taskInstanceMetrics, executor,
+                      taskBackendFactoryToStoreNames.get(taskInstanceName).get(factoryName), loggedStoreDir, jobContext,
+                      containerModel);
+              try {
+                if (future != null) {
+                  future.join(); // Block and restore deleted state before continuing
+                }
+              } catch (Exception e) {
+                String msg = String.format("Unable to recover from DeletedException for task %s.", taskName);
+                throw new SamzaException(msg, e);
+              } finally {
+                updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+              }
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("Error restoring state for task: %s", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex); // wrap in unchecked exception to throw from lambda
+            }
+          }
+
+          // Stop all persistent stores after restoring. Certain persistent stores opened in BulkLoad mode are compacted
+          // on stop, so paralleling stop() also parallelizes their compaction (a time-intensive operation).
+          try {
+            taskRestoreManager.close();
+          } catch (Exception e) {
+            LOG.error("Error closing restore manager for task: {} after {} restore", taskName,
+                ex != null ? "unsuccessful" : "successful", e);
+            // ignore exception from close. container may still be able to continue processing/backups
+            // if restore manager close fails.
+          }
+          return null;
+        });
+        taskRestoreFutures.add(taskRestoreFuture);
+      });
+    });
+
+    return taskRestoreFutures;
+  }
+
+  public static CompletableFuture<Void> restoreTaskStoresAndCreateCheckpointWithGetDeleted(TaskName taskName,
+      Map<TaskName, Checkpoint> taskCheckpoints, CheckpointManager checkpointManager,
+      TaskRestoreManager taskRestoreManager, Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      ExecutorService executor, Set<String> storesToRestore, File loggedStoreBaseDirectory, JobContext jobContext,
+      ContainerModel containerModel) {
+
+    // if taskInstanceMetrics are specified use those for store metrics,
+    // otherwise (in case of StorageRecovery) use a blank MetricsRegistryMap
+    MetricsRegistry metricsRegistry =
+        taskInstanceMetrics.get(taskName) != null ? taskInstanceMetrics.get(taskName).registry()
+            : new MetricsRegistryMap();
+
+    BlobStoreManager blobStoreManager = getBlobStoreManager(config, executor);
+    JobConfig jobConfig = new JobConfig(config);
+    BlobStoreUtil blobStoreUtil =
+        new BlobStoreUtil(blobStoreManager, executor, new BlobStoreConfig(config), null,
+            new BlobStoreRestoreManagerMetrics(metricsRegistry));
+
+    BlobStoreRestoreManager blobStoreRestoreManager = (BlobStoreRestoreManager) taskRestoreManager;
+
+    CheckpointId checkpointId = CheckpointId.create();
+    Map<String, String> oldSCMs = ((CheckpointV2) taskCheckpoints.get(taskName))
+        .getStateCheckpointMarkers().get(BlobStoreStateBackendFactory.class.getName());
+
+    // Returns a single future that guarantees all the following are completed, in this order:
+    // 1. Restore state locally by getting deleted blobs from the blob store.
+    // 2. Create a new Checkpoint from restored state and back it up on the blob store.
+    // 3. Clean up old/deleted Snapshot
+    // 4. Remove TTL from the new Snapshot on the blob store
+    // 5. Write the new checkpoint to checkpoint topic and update taskCheckpoints map
+
+    // 1. restore state with getDeleted flag set to true
+    return blobStoreRestoreManager.restore(true).thenCompose(r -> {
+      // 2. Create and backup new checkpoint on the blob store.
+      CompletableFuture<Map<String, String>> uploadSCMs =
+          createNewCheckpointAndBackupStores(jobContext, containerModel, config, taskName, storesToRestore, checkpointId,
+              loggedStoreBaseDirectory, blobStoreManager, metricsRegistry, executor);
+      CompletableFuture<Void> future = uploadSCMs.thenCompose(scms -> {
+        // 3. Delete prev SnapshotIndex including files/subdirs,
+        Map<String, CompletionStage<Void>> deleteOldSnapshotsFutures =
+            deletePrevSnapshots(blobStoreUtil, oldSCMs, taskName, jobConfig);
+        // 4. Mark new Snapshots to never expire
+        Map<String, CompletionStage<Void>> removeNewSnapshotsTTLFutures =
+            removeTTLNewSnapshots(blobStoreUtil, scms, taskName, jobConfig);
+        // Combined future of delete old snapshots and removeTTL of new snapshots
+        CompletableFuture<Void> removeTTLFuture = CompletableFuture.allOf(FutureUtil.mapToFuture(deleteOldSnapshotsFutures),
+            FutureUtil.mapToFuture(removeNewSnapshotsTTLFutures));
+        // 5. Update taskCheckpoints map with new checkpoint
+        CompletableFuture<Void> checkpointFuture = removeTTLFuture.thenRun(() -> {

Review Comment:
   Updated to clarify each step and their dependency. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] shekhars-li commented on a diff in pull request #1676: [SAMZA-2787] GetDeleted API and Recover from DeletedException

Posted by "shekhars-li (via GitHub)" <gi...@apache.org>.
shekhars-li commented on code in PR #1676:
URL: https://github.com/apache/samza/pull/1676#discussion_r1287895056


##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreBackupManager.java:
##########
@@ -139,8 +139,10 @@ public void init(Checkpoint checkpoint) {
 
     // Note: blocks the caller thread.
     // TODO LOW shesharma exclude stores that are no longer configured during init
+    // NOTE: Always get SnapshotIndex with getDeleted enabled. A failure to get a blob from SnapshotIndex would restart

Review Comment:
   Sorry, the comment isn't accurate. Correcting this. We should pass false here and let the restart/restore recover so that we can get all blobs associated with this snapshot index restored as well. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] shekhars-li commented on a diff in pull request #1676: [SAMZA-2787] GetDeleted API and Recover from DeletedException

Posted by "shekhars-li (via GitHub)" <gi...@apache.org>.
shekhars-li commented on code in PR #1676:
URL: https://github.com/apache/samza/pull/1676#discussion_r1286266903


##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreRestoreManager.java:
##########
@@ -254,8 +263,10 @@ static CompletableFuture<Void> restoreStores(String jobName, String jobId, TaskN
         throw new SamzaException(String.format("Error deleting store directory: %s", storeDir), e);
       }
 
+      // If getDeletedBlobs is enabled - always restore so that we get all the blobs, including the deleted blobs,
+      // immediately restore it locally and backup to create new checkpoint.
       boolean shouldRestore = shouldRestore(taskName.getTaskName(), storeName, dirIndex,
-          storeCheckpointDir, storageConfig, dirDiffUtil);
+          storeCheckpointDir, storageConfig, dirDiffUtil) || getDeletedBlobs;

Review Comment:
   > Any scenarios where shouldRestore returns false but we want to force restore?
   Yes.
   When there is no diff between remote and local snapshot, shouldRestore could return false. However, if the SnapshotIndex was deleted, we want to force restore everything (and then backup, create new checkpoint etc). This is because init() delegates this to restore and only gets the snapshotIndex with getDeleted. It does not actually create a new copy of everything.



##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreRestoreManager.java:
##########
@@ -254,8 +263,10 @@ static CompletableFuture<Void> restoreStores(String jobName, String jobId, TaskN
         throw new SamzaException(String.format("Error deleting store directory: %s", storeDir), e);
       }
 
+      // If getDeletedBlobs is enabled - always restore so that we get all the blobs, including the deleted blobs,
+      // immediately restore it locally and backup to create new checkpoint.
       boolean shouldRestore = shouldRestore(taskName.getTaskName(), storeName, dirIndex,
-          storeCheckpointDir, storageConfig, dirDiffUtil);
+          storeCheckpointDir, storageConfig, dirDiffUtil) || getDeletedBlobs;

Review Comment:
   > Any scenarios where shouldRestore returns false but we want to force restore?
   
   Yes.
   When there is no diff between remote and local snapshot, shouldRestore could return false. However, if the SnapshotIndex was deleted, we want to force restore everything (and then backup, create new checkpoint etc). This is because init() delegates this to restore and only gets the snapshotIndex with getDeleted. It does not actually create a new copy of everything.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] shekhars-li commented on a diff in pull request #1676: [SAMZA-2787] GetDeleted API and Recover from DeletedException

Posted by "shekhars-li (via GitHub)" <gi...@apache.org>.
shekhars-li commented on code in PR #1676:
URL: https://github.com/apache/samza/pull/1676#discussion_r1286270848


##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/util/BlobStoreUtil.java:
##########
@@ -210,6 +216,56 @@ public CompletableFuture<String> putSnapshotIndex(SnapshotIndex snapshotIndex) {
     }, isCauseNonRetriable(), executor, retryPolicyConfig);
   }
 
+  /**
+   * Cleans up a SnapshotIndex by recursively deleting all blobs associated with files/subdirs inside the SnapshotIndex
+   * and finally deletes SnapshotIndex blob itself. This is done by getting the SnapshotIndex first.
+   * @param snapshotIndexBlobId Blob if of SnapshotIndex
+   * @param requestMetadata Metadata of the request
+   */
+  public CompletionStage<Void> cleanSnapshotIndex(String snapshotIndexBlobId, Metadata requestMetadata) {
+    return cleanSnapshotIndex(snapshotIndexBlobId, requestMetadata, false);
+  }
+
+  /**
+   * Cleans up a SnapshotIndex by recursively deleting all blobs associated with files/subdirs inside the SnapshotIndex
+   * and finally deletes SnapshotIndex blob itself. This is done by getting the SnapshotIndex first.
+   * @param snapshotIndexBlobId Blob if of SnapshotIndex
+   * @param requestMetadata Metadata of the request
+   * @param getDeletedBlob Gets SnapshotIndex with getDeleted flag set
+   */
+  public CompletionStage<Void> cleanSnapshotIndex(String snapshotIndexBlobId, Metadata requestMetadata, Boolean getDeletedBlob) {
+    Metadata getSnapshotRequest = new Metadata(Metadata.SNAPSHOT_INDEX_PAYLOAD_PATH, Optional.empty(), requestMetadata.getJobName(),
+        requestMetadata.getJobId(), requestMetadata.getTaskName(), requestMetadata.getStoreName());
+    SnapshotIndex snapshotIndex = getSnapshotIndex(snapshotIndexBlobId, getSnapshotRequest, getDeletedBlob).join();
+    return cleanSnapshotIndex(snapshotIndexBlobId, snapshotIndex, requestMetadata);
+  }
+
+  /**
+   * Cleans up a SnapshotIndex by recursively deleting all blobs associated with files/subdirs inside the SnapshotIndex
+   * and finally deletes SnapshotIndex blob itself.
+   * @param snapshotIndexBlobId Blob if of SnapshotIndex
+   * @param snapshotIndex SnapshotIndex to delete
+   * @param requestMetadata Metadata of the request
+   */
+  public CompletionStage<Void> cleanSnapshotIndex(String snapshotIndexBlobId, SnapshotIndex snapshotIndex, Metadata requestMetadata) {
+    DirIndex dirIndex = snapshotIndex.getDirIndex();
+    CompletionStage<Void> storeDeletionFuture =
+        cleanUpDir(dirIndex, requestMetadata) // delete files and sub-dirs previously marked for removal
+            .thenComposeAsync(v ->
+                deleteDir(dirIndex, requestMetadata), executor) // deleted files and dirs still present
+            .thenComposeAsync(v -> deleteSnapshotIndexBlob(snapshotIndexBlobId, requestMetadata), executor) // delete the snapshot index blob
+            .exceptionally(ex -> {
+              if (ex instanceof DeletedException) {

Review Comment:
   Corrected that. Thanks



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] shekhars-li commented on a diff in pull request #1676: [SAMZA-2787] GetDeleted API and Recover from DeletedException

Posted by "shekhars-li (via GitHub)" <gi...@apache.org>.
shekhars-li commented on code in PR #1676:
URL: https://github.com/apache/samza/pull/1676#discussion_r1275518474


##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java:
##########
@@ -301,64 +300,14 @@ private void restoreStores() throws InterruptedException {
               samzaContainerMetrics, taskInstanceMetrics, taskInstanceCollectors, serdes,
               loggedStoreBaseDirectory, nonLoggedStoreBaseDirectory, config, clock);
       taskRestoreManagers.put(taskName, taskStoreRestoreManagers);
+      taskBackendFactoryToStoreNames.put(taskName, backendFactoryToStoreNames);
     });
 
-    // Initialize each TaskStorageManager
-    taskRestoreManagers.forEach((taskName, restoreManagers) ->
-        restoreManagers.forEach((factoryName, taskRestoreManager) ->
-            taskRestoreManager.init(taskCheckpoints.get(taskName))
-        )
-    );
-
-    // Start each store consumer once.
-    // Note: These consumers are per system and only changelog system store consumers will be started.
-    // Some TaskRestoreManagers may not require the consumer to to be started, but due to the agnostic nature of
-    // ContainerStorageManager we always start the changelog consumer here in case it is required
-    this.storeConsumers.values().stream().distinct().forEach(SystemConsumer::start);
-
-    List<Future<Void>> taskRestoreFutures = new ArrayList<>();
-
-    // Submit restore callable for each taskInstance
-    taskRestoreManagers.forEach((taskInstance, restoreManagersMap) -> {
-      // Submit for each restore factory
-      restoreManagersMap.forEach((factoryName, taskRestoreManager) -> {
-        long startTime = System.currentTimeMillis();
-        String taskName = taskInstance.getTaskName();
-        LOG.info("Starting restore for state for task: {}", taskName);
-        CompletableFuture<Void> restoreFuture = taskRestoreManager.restore().handle((res, ex) -> {
-          // Stop all persistent stores after restoring. Certain persistent stores opened in BulkLoad mode are compacted
-          // on stop, so paralleling stop() also parallelizes their compaction (a time-intensive operation).
-          try {
-            taskRestoreManager.close();
-          } catch (Exception e) {
-            LOG.error("Error closing restore manager for task: {} after {} restore",
-                taskName, ex != null ? "unsuccessful" : "successful", e);
-            // ignore exception from close. container may still be be able to continue processing/backups
-            // if restore manager close fails.
-          }
-
-          long timeToRestore = System.currentTimeMillis() - startTime;
-          if (samzaContainerMetrics != null) {
-            Gauge taskGauge = samzaContainerMetrics.taskStoreRestorationMetrics().getOrDefault(taskInstance, null);
-
-            if (taskGauge != null) {
-              taskGauge.set(timeToRestore);
-            }
-          }
-
-          if (ex != null) {
-            // log and rethrow exception to communicate restore failure
-            String msg = String.format("Error restoring state for task: %s", taskName);
-            LOG.error(msg, ex);
-            throw new SamzaException(msg, ex); // wrap in unchecked exception to throw from lambda
-          } else {
-            return null;
-          }
-        });
-
-        taskRestoreFutures.add(restoreFuture);
-      });
-    });
+    // Init all taskRestores and if successful, create a future for restores for each task
+    List<Future<Void>> taskRestoreFutures =

Review Comment:
   @prateekm Newly created checkpoints are reflected in the TaskCheckpoints map. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] prateekm commented on a diff in pull request #1676: GetDeleted API and Recover from DeletedException

Posted by "prateekm (via GitHub)" <gi...@apache.org>.
prateekm commented on code in PR #1676:
URL: https://github.com/apache/samza/pull/1676#discussion_r1273936574


##########
samza-core/src/main/java/org/apache/samza/storage/TaskStorageCommitManager.java:
##########
@@ -93,14 +93,18 @@ public TaskStorageCommitManager(TaskName taskName, Map<String, TaskBackupManager
     this.metrics = metrics;
   }
 
-  public void init() {
+  public void init(Checkpoint checkpoint) {
     // Assuming that container storage manager has already started and created to stores
     storageEngines = containerStorageManager.getAllStores(taskName);
     if (checkpointManager != null) {
-      Checkpoint checkpoint = checkpointManager.readLastCheckpoint(taskName);
+      // In case of standby containers

Review Comment:
   Minor: conditions can be simplified. checkpointManager != null only needs to be checked if checkpoint == null.
   Also prefer not reassigning input params, assign to finalCheckpoint instead.



##########
samza-api/src/main/java/org/apache/samza/storage/blobstore/BlobStoreManager.java:
##########
@@ -49,11 +49,12 @@ public interface BlobStoreManager {
    * @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
+   * @param getDeletedBlob Flag to indicate if get should try to get a blob marked for deletion but not yet compacted
    * @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);
+  CompletionStage<Void> get(String id, OutputStream outputStream, Metadata metadata, Boolean getDeletedBlob);

Review Comment:
   Minor: boolean, to avoid accidentally passing null values



##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreRestoreManager.java:
##########
@@ -119,13 +119,18 @@ public BlobStoreRestoreManager(TaskModel taskModel, ExecutorService restoreExecu
 
   @Override
   public void init(Checkpoint checkpoint) {
+    init(checkpoint, false);
+  }
+
+  public void init(Checkpoint checkpoint, Boolean getDeletedBlob) {

Review Comment:
   Do we need both init variants / why do we need both to be public? 



##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreRestoreManager.java:
##########
@@ -153,6 +158,10 @@ public CompletableFuture<Void> restore() {
         storageConfig, metrics, storageManagerUtil, blobStoreUtil, dirDiffUtil, executor);
   }
 
+  public CompletableFuture<Void> restore(Boolean restoreDeleted) {

Review Comment:
   Add javadoc to public method. Why do we need both variants of restore?



##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/util/BlobStoreUtil.java:
##########
@@ -162,15 +167,7 @@ public Map<String, Pair<String, SnapshotIndex>> getStoreSnapshotIndexes(
     }
 
     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();
+      return FutureUtil.toFutureOfMap(storeSnapshotIndexFutures).join();

Review Comment:
   Not obvious to me: can you explain the difference in error handling behavior before / after? What does the predicate param to toFutureOfMap do?



##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreRestoreManager.java:
##########
@@ -329,11 +340,11 @@ static boolean shouldRestore(String taskName, String storeName, DirIndex dirInde
   @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) {
+      DirDiffUtil dirDiffUtil, BlobStoreRestoreManagerMetrics metrics, ExecutorService executor, Boolean getDeleted) {

Review Comment:
   boolean



##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreRestoreManager.java:
##########
@@ -119,13 +119,18 @@ public BlobStoreRestoreManager(TaskModel taskModel, ExecutorService restoreExecu
 
   @Override
   public void init(Checkpoint checkpoint) {
+    init(checkpoint, false);

Review Comment:
   Why false? Document inline.



##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/util/BlobStoreUtil.java:
##########
@@ -232,6 +288,15 @@ public CompletionStage<Void> deleteSnapshotIndexBlob(String snapshotIndexBlobId,
    * @return A future that completes when all the async downloads completes
    */
   public CompletableFuture<Void> restoreDir(File baseDir, DirIndex dirIndex, Metadata metadata) {
+    return restoreDir(baseDir, dirIndex, metadata, false);
+  }
+
+  /**
+   * Non-blocking restore of a {@link SnapshotIndex} to local store by downloading all the files and sub-dirs associated
+   * with this remote snapshot. getDeletedFiles flag sets whether to attempt a get for deletedFiles or not.

Review Comment:
   Minor: update doc for getDeletedFile flag (explain why / when to use, not "getDeletedFiles gets deleted files" etc.)
   



##########
samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala:
##########
@@ -981,7 +981,7 @@ class SamzaContainer(
     }
   }
 
-  def startStores {
+  def startStores: util.Map[TaskName, Checkpoint] = {

Review Comment:
   Add documentation for what the return value represents.



##########
samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala:
##########
@@ -993,10 +993,14 @@ class SamzaContainer(
     })
   }
 
-  def startTask {
+  def startTask(taskCheckpoints: util.Map[TaskName, Checkpoint]) {

Review Comment:
   Add documentation here and later on for what the param represents.
   
   Esp. clarify relationship to startpoints.



##########
samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala:
##########
@@ -149,31 +149,33 @@ class TaskInstance(
     }
   }
 
-  def initTask {
+  def initTask(lastTaskCheckpoint: Checkpoint) {
     initCaughtUpMapping()
 
+    val isStandByTask = taskModel.getTaskMode == TaskMode.Standby
+
     if (commitManager != null) {
       debug("Starting commit manager for taskName: %s" format taskName)
-
-      commitManager.init()
+      commitManager.init(if (isStandByTask) null else lastTaskCheckpoint)

Review Comment:
   Minor: Prefer not inlining expresions as params. Assign explicitly.



##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java:
##########
@@ -208,9 +205,9 @@ public ContainerStorageManager(
   }
 
 
-  public void start() throws SamzaException, InterruptedException {
+  public Map<TaskName, Checkpoint> start() throws SamzaException, InterruptedException {

Review Comment:
   Document return value.



##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreRestoreManager.java:
##########
@@ -153,6 +158,10 @@ public CompletableFuture<Void> restore() {
         storageConfig, metrics, storageManagerUtil, blobStoreUtil, dirDiffUtil, executor);
   }
 
+  public CompletableFuture<Void> restore(Boolean restoreDeleted) {
+    return restoreStores(jobName, jobId, taskModel.getTaskName(), storesToRestore, prevStoreSnapshotIndexes,
+        loggedBaseDir, storageConfig, metrics, storageManagerUtil, blobStoreUtil, dirDiffUtil, executor, restoreDeleted);
+  }

Review Comment:
   Newline after.



##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java:
##########
@@ -301,64 +300,14 @@ private void restoreStores() throws InterruptedException {
               samzaContainerMetrics, taskInstanceMetrics, taskInstanceCollectors, serdes,
               loggedStoreBaseDirectory, nonLoggedStoreBaseDirectory, config, clock);
       taskRestoreManagers.put(taskName, taskStoreRestoreManagers);
+      taskBackendFactoryToStoreNames.put(taskName, backendFactoryToStoreNames);
     });
 
-    // Initialize each TaskStorageManager
-    taskRestoreManagers.forEach((taskName, restoreManagers) ->
-        restoreManagers.forEach((factoryName, taskRestoreManager) ->
-            taskRestoreManager.init(taskCheckpoints.get(taskName))
-        )
-    );
-
-    // Start each store consumer once.
-    // Note: These consumers are per system and only changelog system store consumers will be started.
-    // Some TaskRestoreManagers may not require the consumer to to be started, but due to the agnostic nature of
-    // ContainerStorageManager we always start the changelog consumer here in case it is required
-    this.storeConsumers.values().stream().distinct().forEach(SystemConsumer::start);
-
-    List<Future<Void>> taskRestoreFutures = new ArrayList<>();
-
-    // Submit restore callable for each taskInstance
-    taskRestoreManagers.forEach((taskInstance, restoreManagersMap) -> {
-      // Submit for each restore factory
-      restoreManagersMap.forEach((factoryName, taskRestoreManager) -> {
-        long startTime = System.currentTimeMillis();
-        String taskName = taskInstance.getTaskName();
-        LOG.info("Starting restore for state for task: {}", taskName);
-        CompletableFuture<Void> restoreFuture = taskRestoreManager.restore().handle((res, ex) -> {
-          // Stop all persistent stores after restoring. Certain persistent stores opened in BulkLoad mode are compacted
-          // on stop, so paralleling stop() also parallelizes their compaction (a time-intensive operation).
-          try {
-            taskRestoreManager.close();
-          } catch (Exception e) {
-            LOG.error("Error closing restore manager for task: {} after {} restore",
-                taskName, ex != null ? "unsuccessful" : "successful", e);
-            // ignore exception from close. container may still be be able to continue processing/backups
-            // if restore manager close fails.
-          }
-
-          long timeToRestore = System.currentTimeMillis() - startTime;
-          if (samzaContainerMetrics != null) {
-            Gauge taskGauge = samzaContainerMetrics.taskStoreRestorationMetrics().getOrDefault(taskInstance, null);
-
-            if (taskGauge != null) {
-              taskGauge.set(timeToRestore);
-            }
-          }
-
-          if (ex != null) {
-            // log and rethrow exception to communicate restore failure
-            String msg = String.format("Error restoring state for task: %s", taskName);
-            LOG.error(msg, ex);
-            throw new SamzaException(msg, ex); // wrap in unchecked exception to throw from lambda
-          } else {
-            return null;
-          }
-        });
-
-        taskRestoreFutures.add(restoreFuture);
-      });
-    });
+    // Init all taskRestores and if successful, create a future for restores for each task

Review Comment:
   "Init all *tasks stores*, and if successful, (concurrently?) restore each task store".
   
   Minor: Generally should not say "create a future for" since Future is just the return type, not the main action. Same reason you'd say "get members" and not "create a list for members".



##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreRestoreManager.java:
##########
@@ -212,9 +213,17 @@ static CompletableFuture<Void> restoreStores(String jobName, String jobId, TaskN
       File loggedBaseDir, StorageConfig storageConfig, BlobStoreRestoreManagerMetrics metrics,
       StorageManagerUtil storageManagerUtil, BlobStoreUtil blobStoreUtil, DirDiffUtil dirDiffUtil,
       ExecutorService executor) {
+    return restoreStores(jobName, jobId, taskName, storesToRestore, prevStoreSnapshotIndexes, loggedBaseDir, storageConfig,
+        metrics, storageManagerUtil, blobStoreUtil, dirDiffUtil, executor, false);
+  }
+
+  public static CompletableFuture<Void> restoreStores(String jobName, String jobId, TaskName taskName, Set<String> storesToRestore,

Review Comment:
   Why 2 variants? Previous method wasn't public. Why does this need to be?
   



##########
samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala:
##########
@@ -149,31 +149,33 @@ class TaskInstance(
     }
   }
 
-  def initTask {
+  def initTask(lastTaskCheckpoint: Checkpoint) {

Review Comment:
   Also document this is nullable (or better, pass a scala Option instead)



##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManagerUtil.java:
##########
@@ -411,4 +438,292 @@ public static Set<String> getSideInputStoreNames(
     }
     return sideInputStores;
   }
+
+  public static List<Future<Void>> initAndRestoreTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, SamzaContainerMetrics samzaContainerMetrics,
+      CheckpointManager checkpointManager, JobContext jobContext, ContainerModel containerModel,
+      Map<TaskName, Checkpoint> taskCheckpoints, Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames,
+      Config config, ExecutorService executor, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      File loggerStoreDir, Map<String, SystemConsumer> storeConsumers) {
+
+    Set<String> forceRestoreTasks = new HashSet<>();
+    // Initialize each TaskStorageManager.
+    taskRestoreManagers.forEach((taskName, restoreManagers) ->
+        restoreManagers.forEach((factoryName, taskRestoreManager) -> {
+          try {
+            taskRestoreManager.init(taskCheckpoints.get(taskName));
+          } catch (SamzaException ex) {
+            if (isUnwrappedExceptionDeletedException(ex) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+              // Get deleted SnapshotIndex blob with GetDeleted and mark the task to be restored with GetDeleted as well.
+              // this ensures that the restore downloads the snapshot, recreates a new snapshot, uploads it to blob store
+              // and creates a new checkpoint.
+              ((BlobStoreRestoreManager) taskRestoreManager).init(taskCheckpoints.get(taskName), true);
+              forceRestoreTasks.add(taskName.getTaskName());
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("init failed for task: %s with GetDeleted set to true", taskName);

Review Comment:
   Failed for restore manager, not for task.
   



##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreRestoreManager.java:
##########
@@ -254,8 +263,10 @@ static CompletableFuture<Void> restoreStores(String jobName, String jobId, TaskN
         throw new SamzaException(String.format("Error deleting store directory: %s", storeDir), e);
       }
 
+      // If getDeletedBlobs is enabled - always restore so that we get all the blobs, including the deleted blobs,
+      // immediately restore it locally and backup to create new checkpoint.
       boolean shouldRestore = shouldRestore(taskName.getTaskName(), storeName, dirIndex,
-          storeCheckpointDir, storageConfig, dirDiffUtil);
+          storeCheckpointDir, storageConfig, dirDiffUtil) || getDeletedBlobs;

Review Comment:
   Move getDeletedBlob check to shouldRestore and update javadoc.



##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/util/BlobStoreUtil.java:
##########
@@ -114,6 +114,11 @@ public BlobStoreUtil(BlobStoreManager blobStoreManager, ExecutorService executor
    */
   public Map<String, Pair<String, SnapshotIndex>> getStoreSnapshotIndexes(
       String jobName, String jobId, String taskName, Checkpoint checkpoint, Set<String> storesToBackupOrRestore) {
+    return getStoreSnapshotIndexes(jobName, jobId, taskName, checkpoint, storesToBackupOrRestore, false);
+  }
+
+  public Map<String, Pair<String, SnapshotIndex>> getStoreSnapshotIndexes(String jobName, String jobId, String taskName,

Review Comment:
   Same as other comments re: variants.



##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/util/BlobStoreUtil.java:
##########
@@ -210,6 +216,56 @@ public CompletableFuture<String> putSnapshotIndex(SnapshotIndex snapshotIndex) {
     }, isCauseNonRetriable(), executor, retryPolicyConfig);
   }
 
+  /**
+   * Cleans up a SnapshotIndex by recursively deleting all blobs associated with files/subdirs inside the SnapshotIndex
+   * and finally deletes SnapshotIndex blob itself. This is done by getting the SnapshotIndex first.
+   * @param snapshotIndexBlobId Blob if of SnapshotIndex
+   * @param requestMetadata Metadata of the request
+   */
+  public CompletionStage<Void> cleanSnapshotIndex(String snapshotIndexBlobId, Metadata requestMetadata) {

Review Comment:
   Same as other comments re: variants.



##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/util/BlobStoreUtil.java:
##########
@@ -183,11 +180,20 @@ public Map<String, Pair<String, SnapshotIndex>> getStoreSnapshotIndexes(
    * @return a Future containing the {@link SnapshotIndex}
    */
   public CompletableFuture<SnapshotIndex> getSnapshotIndex(String blobId, Metadata metadata) {
+    return getSnapshotIndex(blobId, metadata, false);
+  }
+
+  /**
+   * 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, Boolean getDeletedBlob) {

Review Comment:
   Same as other comments re: variants.



##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/util/BlobStoreUtil.java:
##########
@@ -210,6 +216,56 @@ public CompletableFuture<String> putSnapshotIndex(SnapshotIndex snapshotIndex) {
     }, isCauseNonRetriable(), executor, retryPolicyConfig);
   }
 
+  /**
+   * Cleans up a SnapshotIndex by recursively deleting all blobs associated with files/subdirs inside the SnapshotIndex
+   * and finally deletes SnapshotIndex blob itself. This is done by getting the SnapshotIndex first.
+   * @param snapshotIndexBlobId Blob if of SnapshotIndex
+   * @param requestMetadata Metadata of the request
+   */
+  public CompletionStage<Void> cleanSnapshotIndex(String snapshotIndexBlobId, Metadata requestMetadata) {
+    return cleanSnapshotIndex(snapshotIndexBlobId, requestMetadata, false);
+  }
+
+  /**
+   * Cleans up a SnapshotIndex by recursively deleting all blobs associated with files/subdirs inside the SnapshotIndex
+   * and finally deletes SnapshotIndex blob itself. This is done by getting the SnapshotIndex first.
+   * @param snapshotIndexBlobId Blob if of SnapshotIndex
+   * @param requestMetadata Metadata of the request
+   * @param getDeletedBlob Gets SnapshotIndex with getDeleted flag set
+   */
+  public CompletionStage<Void> cleanSnapshotIndex(String snapshotIndexBlobId, Metadata requestMetadata, Boolean getDeletedBlob) {
+    Metadata getSnapshotRequest = new Metadata(Metadata.SNAPSHOT_INDEX_PAYLOAD_PATH, Optional.empty(), requestMetadata.getJobName(),
+        requestMetadata.getJobId(), requestMetadata.getTaskName(), requestMetadata.getStoreName());
+    SnapshotIndex snapshotIndex = getSnapshotIndex(snapshotIndexBlobId, getSnapshotRequest, getDeletedBlob).join();
+    return cleanSnapshotIndex(snapshotIndexBlobId, snapshotIndex, requestMetadata);
+  }
+
+  /**
+   * Cleans up a SnapshotIndex by recursively deleting all blobs associated with files/subdirs inside the SnapshotIndex
+   * and finally deletes SnapshotIndex blob itself.
+   * @param snapshotIndexBlobId Blob if of SnapshotIndex
+   * @param snapshotIndex SnapshotIndex to delete
+   * @param requestMetadata Metadata of the request
+   */
+  public CompletionStage<Void> cleanSnapshotIndex(String snapshotIndexBlobId, SnapshotIndex snapshotIndex, Metadata requestMetadata) {
+    DirIndex dirIndex = snapshotIndex.getDirIndex();
+    CompletionStage<Void> storeDeletionFuture =
+        cleanUpDir(dirIndex, requestMetadata) // delete files and sub-dirs previously marked for removal
+            .thenComposeAsync(v ->
+                deleteDir(dirIndex, requestMetadata), executor) // deleted files and dirs still present
+            .thenComposeAsync(v -> deleteSnapshotIndexBlob(snapshotIndexBlobId, requestMetadata), executor) // delete the snapshot index blob
+            .exceptionally(ex -> {
+              if (ex instanceof DeletedException) {

Review Comment:
   Document error handling behavior (explain what it should do / is trying to do).
   
   Do you need to unwrap CompletedException here?



##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/util/BlobStoreUtil.java:
##########
@@ -210,6 +216,56 @@ public CompletableFuture<String> putSnapshotIndex(SnapshotIndex snapshotIndex) {
     }, isCauseNonRetriable(), executor, retryPolicyConfig);
   }
 
+  /**
+   * Cleans up a SnapshotIndex by recursively deleting all blobs associated with files/subdirs inside the SnapshotIndex
+   * and finally deletes SnapshotIndex blob itself. This is done by getting the SnapshotIndex first.
+   * @param snapshotIndexBlobId Blob if of SnapshotIndex
+   * @param requestMetadata Metadata of the request
+   */
+  public CompletionStage<Void> cleanSnapshotIndex(String snapshotIndexBlobId, Metadata requestMetadata) {
+    return cleanSnapshotIndex(snapshotIndexBlobId, requestMetadata, false);
+  }
+
+  /**
+   * Cleans up a SnapshotIndex by recursively deleting all blobs associated with files/subdirs inside the SnapshotIndex
+   * and finally deletes SnapshotIndex blob itself. This is done by getting the SnapshotIndex first.
+   * @param snapshotIndexBlobId Blob if of SnapshotIndex
+   * @param requestMetadata Metadata of the request
+   * @param getDeletedBlob Gets SnapshotIndex with getDeleted flag set
+   */
+  public CompletionStage<Void> cleanSnapshotIndex(String snapshotIndexBlobId, Metadata requestMetadata, Boolean getDeletedBlob) {
+    Metadata getSnapshotRequest = new Metadata(Metadata.SNAPSHOT_INDEX_PAYLOAD_PATH, Optional.empty(), requestMetadata.getJobName(),
+        requestMetadata.getJobId(), requestMetadata.getTaskName(), requestMetadata.getStoreName());
+    SnapshotIndex snapshotIndex = getSnapshotIndex(snapshotIndexBlobId, getSnapshotRequest, getDeletedBlob).join();
+    return cleanSnapshotIndex(snapshotIndexBlobId, snapshotIndex, requestMetadata);
+  }
+
+  /**
+   * Cleans up a SnapshotIndex by recursively deleting all blobs associated with files/subdirs inside the SnapshotIndex
+   * and finally deletes SnapshotIndex blob itself.
+   * @param snapshotIndexBlobId Blob if of SnapshotIndex
+   * @param snapshotIndex SnapshotIndex to delete
+   * @param requestMetadata Metadata of the request
+   */
+  public CompletionStage<Void> cleanSnapshotIndex(String snapshotIndexBlobId, SnapshotIndex snapshotIndex, Metadata requestMetadata) {
+    DirIndex dirIndex = snapshotIndex.getDirIndex();
+    CompletionStage<Void> storeDeletionFuture =
+        cleanUpDir(dirIndex, requestMetadata) // delete files and sub-dirs previously marked for removal
+            .thenComposeAsync(v ->
+                deleteDir(dirIndex, requestMetadata), executor) // deleted files and dirs still present
+            .thenComposeAsync(v -> deleteSnapshotIndexBlob(snapshotIndexBlobId, requestMetadata), executor) // delete the snapshot index blob
+            .exceptionally(ex -> {
+              if (ex instanceof DeletedException) {
+                LOG.warn("DeletedException received on trying to clean up SnapshotIndex {}. Ignoring the error.",
+                    snapshotIndexBlobId);
+                return null;
+              }
+              String msg = String.format("Error deleting/cleaning up SnapshotIndex: %s", snapshotIndexBlobId);

Review Comment:
   Same as above, add more context.



##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/util/BlobStoreUtil.java:
##########
@@ -210,6 +216,56 @@ public CompletableFuture<String> putSnapshotIndex(SnapshotIndex snapshotIndex) {
     }, isCauseNonRetriable(), executor, retryPolicyConfig);
   }
 
+  /**
+   * Cleans up a SnapshotIndex by recursively deleting all blobs associated with files/subdirs inside the SnapshotIndex
+   * and finally deletes SnapshotIndex blob itself. This is done by getting the SnapshotIndex first.
+   * @param snapshotIndexBlobId Blob if of SnapshotIndex
+   * @param requestMetadata Metadata of the request
+   */
+  public CompletionStage<Void> cleanSnapshotIndex(String snapshotIndexBlobId, Metadata requestMetadata) {
+    return cleanSnapshotIndex(snapshotIndexBlobId, requestMetadata, false);
+  }
+
+  /**
+   * Cleans up a SnapshotIndex by recursively deleting all blobs associated with files/subdirs inside the SnapshotIndex
+   * and finally deletes SnapshotIndex blob itself. This is done by getting the SnapshotIndex first.
+   * @param snapshotIndexBlobId Blob if of SnapshotIndex
+   * @param requestMetadata Metadata of the request
+   * @param getDeletedBlob Gets SnapshotIndex with getDeleted flag set
+   */
+  public CompletionStage<Void> cleanSnapshotIndex(String snapshotIndexBlobId, Metadata requestMetadata, Boolean getDeletedBlob) {

Review Comment:
   Minor: boolean.
   



##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/util/BlobStoreUtil.java:
##########
@@ -210,6 +216,56 @@ public CompletableFuture<String> putSnapshotIndex(SnapshotIndex snapshotIndex) {
     }, isCauseNonRetriable(), executor, retryPolicyConfig);
   }
 
+  /**
+   * Cleans up a SnapshotIndex by recursively deleting all blobs associated with files/subdirs inside the SnapshotIndex
+   * and finally deletes SnapshotIndex blob itself. This is done by getting the SnapshotIndex first.
+   * @param snapshotIndexBlobId Blob if of SnapshotIndex
+   * @param requestMetadata Metadata of the request
+   */
+  public CompletionStage<Void> cleanSnapshotIndex(String snapshotIndexBlobId, Metadata requestMetadata) {
+    return cleanSnapshotIndex(snapshotIndexBlobId, requestMetadata, false);
+  }
+
+  /**
+   * Cleans up a SnapshotIndex by recursively deleting all blobs associated with files/subdirs inside the SnapshotIndex
+   * and finally deletes SnapshotIndex blob itself. This is done by getting the SnapshotIndex first.
+   * @param snapshotIndexBlobId Blob if of SnapshotIndex
+   * @param requestMetadata Metadata of the request
+   * @param getDeletedBlob Gets SnapshotIndex with getDeleted flag set

Review Comment:
   Minor: document behavior, not impl. E.g. "determines whether to attempt to get deleted snapshot index blobs or not." or something.



##########
samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala:
##########
@@ -149,31 +149,33 @@ class TaskInstance(
     }
   }
 
-  def initTask {
+  def initTask(lastTaskCheckpoint: Checkpoint) {
     initCaughtUpMapping()
 
+    val isStandByTask = taskModel.getTaskMode == TaskMode.Standby
+
     if (commitManager != null) {
       debug("Starting commit manager for taskName: %s" format taskName)
-
-      commitManager.init()
+      commitManager.init(if (isStandByTask) null else lastTaskCheckpoint)

Review Comment:
   Document what this param is and its relationship to startpoints in CommitManager interface.



##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/util/BlobStoreUtil.java:
##########
@@ -210,6 +216,56 @@ public CompletableFuture<String> putSnapshotIndex(SnapshotIndex snapshotIndex) {
     }, isCauseNonRetriable(), executor, retryPolicyConfig);
   }
 
+  /**
+   * Cleans up a SnapshotIndex by recursively deleting all blobs associated with files/subdirs inside the SnapshotIndex
+   * and finally deletes SnapshotIndex blob itself. This is done by getting the SnapshotIndex first.
+   * @param snapshotIndexBlobId Blob if of SnapshotIndex
+   * @param requestMetadata Metadata of the request
+   */
+  public CompletionStage<Void> cleanSnapshotIndex(String snapshotIndexBlobId, Metadata requestMetadata) {
+    return cleanSnapshotIndex(snapshotIndexBlobId, requestMetadata, false);
+  }
+
+  /**
+   * Cleans up a SnapshotIndex by recursively deleting all blobs associated with files/subdirs inside the SnapshotIndex
+   * and finally deletes SnapshotIndex blob itself. This is done by getting the SnapshotIndex first.
+   * @param snapshotIndexBlobId Blob if of SnapshotIndex
+   * @param requestMetadata Metadata of the request
+   * @param getDeletedBlob Gets SnapshotIndex with getDeleted flag set
+   */
+  public CompletionStage<Void> cleanSnapshotIndex(String snapshotIndexBlobId, Metadata requestMetadata, Boolean getDeletedBlob) {
+    Metadata getSnapshotRequest = new Metadata(Metadata.SNAPSHOT_INDEX_PAYLOAD_PATH, Optional.empty(), requestMetadata.getJobName(),
+        requestMetadata.getJobId(), requestMetadata.getTaskName(), requestMetadata.getStoreName());
+    SnapshotIndex snapshotIndex = getSnapshotIndex(snapshotIndexBlobId, getSnapshotRequest, getDeletedBlob).join();
+    return cleanSnapshotIndex(snapshotIndexBlobId, snapshotIndex, requestMetadata);
+  }
+
+  /**
+   * Cleans up a SnapshotIndex by recursively deleting all blobs associated with files/subdirs inside the SnapshotIndex
+   * and finally deletes SnapshotIndex blob itself.
+   * @param snapshotIndexBlobId Blob if of SnapshotIndex

Review Comment:
   Minor: Blob id (typo)



##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/util/BlobStoreUtil.java:
##########
@@ -210,6 +216,56 @@ public CompletableFuture<String> putSnapshotIndex(SnapshotIndex snapshotIndex) {
     }, isCauseNonRetriable(), executor, retryPolicyConfig);
   }
 
+  /**
+   * Cleans up a SnapshotIndex by recursively deleting all blobs associated with files/subdirs inside the SnapshotIndex
+   * and finally deletes SnapshotIndex blob itself. This is done by getting the SnapshotIndex first.
+   * @param snapshotIndexBlobId Blob if of SnapshotIndex
+   * @param requestMetadata Metadata of the request
+   */
+  public CompletionStage<Void> cleanSnapshotIndex(String snapshotIndexBlobId, Metadata requestMetadata) {
+    return cleanSnapshotIndex(snapshotIndexBlobId, requestMetadata, false);
+  }
+
+  /**
+   * Cleans up a SnapshotIndex by recursively deleting all blobs associated with files/subdirs inside the SnapshotIndex
+   * and finally deletes SnapshotIndex blob itself. This is done by getting the SnapshotIndex first.
+   * @param snapshotIndexBlobId Blob if of SnapshotIndex
+   * @param requestMetadata Metadata of the request
+   * @param getDeletedBlob Gets SnapshotIndex with getDeleted flag set
+   */
+  public CompletionStage<Void> cleanSnapshotIndex(String snapshotIndexBlobId, Metadata requestMetadata, Boolean getDeletedBlob) {
+    Metadata getSnapshotRequest = new Metadata(Metadata.SNAPSHOT_INDEX_PAYLOAD_PATH, Optional.empty(), requestMetadata.getJobName(),
+        requestMetadata.getJobId(), requestMetadata.getTaskName(), requestMetadata.getStoreName());
+    SnapshotIndex snapshotIndex = getSnapshotIndex(snapshotIndexBlobId, getSnapshotRequest, getDeletedBlob).join();
+    return cleanSnapshotIndex(snapshotIndexBlobId, snapshotIndex, requestMetadata);
+  }
+
+  /**
+   * Cleans up a SnapshotIndex by recursively deleting all blobs associated with files/subdirs inside the SnapshotIndex
+   * and finally deletes SnapshotIndex blob itself.
+   * @param snapshotIndexBlobId Blob if of SnapshotIndex
+   * @param snapshotIndex SnapshotIndex to delete
+   * @param requestMetadata Metadata of the request
+   */
+  public CompletionStage<Void> cleanSnapshotIndex(String snapshotIndexBlobId, SnapshotIndex snapshotIndex, Metadata requestMetadata) {
+    DirIndex dirIndex = snapshotIndex.getDirIndex();
+    CompletionStage<Void> storeDeletionFuture =
+        cleanUpDir(dirIndex, requestMetadata) // delete files and sub-dirs previously marked for removal
+            .thenComposeAsync(v ->
+                deleteDir(dirIndex, requestMetadata), executor) // deleted files and dirs still present
+            .thenComposeAsync(v -> deleteSnapshotIndexBlob(snapshotIndexBlobId, requestMetadata), executor) // delete the snapshot index blob
+            .exceptionally(ex -> {
+              if (ex instanceof DeletedException) {
+                LOG.warn("DeletedException received on trying to clean up SnapshotIndex {}. Ignoring the error.",

Review Comment:
   Any more information we need to log, e.g. storename / taskname etc?



##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/util/BlobStoreUtil.java:
##########
@@ -397,10 +462,11 @@ public CompletionStage<Void> cleanUpDir(DirIndex dirIndex, Metadata metadata) {
    * @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
+   * @param getDeletedFiles Flag that indicates whether to try to get Deleted (but not yet compacted) files.
    * @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) {
+  CompletableFuture<Void> getFile(List<FileBlob> fileBlobs, File fileToRestore, Metadata requestMetadata, Boolean getDeletedFiles) {

Review Comment:
   Minor: boolean



##########
samza-core/src/main/java/org/apache/samza/util/FutureUtil.java:
##########
@@ -139,6 +139,17 @@ public static <K, V> CompletableFuture<Map<K, V>> toFutureOfMap(
     });
   }
 
+  /**
+   * Helper method to convert: {@code Map<K, CompletionStage<Void>>}
+   * to:                       {@code CompletableFuture<Void>}
+   *
+   * Returns a future that completes when all value futures complete.
+   * Returned future completes exceptionally if any of the value futures complete exceptionally.
+   */
+  public static CompletableFuture<Void> mapToFuture(Map<String, CompletionStage<Void>> map) {

Review Comment:
   Does this need to be different from method above?



##########
samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala:
##########
@@ -149,31 +149,33 @@ class TaskInstance(
     }
   }
 
-  def initTask {
+  def initTask(lastTaskCheckpoint: Checkpoint) {
     initCaughtUpMapping()
 
+    val isStandByTask = taskModel.getTaskMode == TaskMode.Standby
+
     if (commitManager != null) {
       debug("Starting commit manager for taskName: %s" format taskName)
-
-      commitManager.init()
+      commitManager.init(if (isStandByTask) null else lastTaskCheckpoint)
     } else {
       debug("Skipping commit manager initialization for taskName: %s" format taskName)
     }
 
-    if (offsetManager != null) {
-      val checkpoint = offsetManager.getLastTaskCheckpoint(taskName)
-      // Only required for checkpointV2
-      if (checkpoint != null && checkpoint.getVersion == 2) {
-        val checkpointV2 = checkpoint.asInstanceOf[CheckpointV2]
-        // call cleanUp on backup managers in case the container previously failed during commit
-        // before completing this step
-
-        // WARNING: cleanUp is NOT optional with blob stores since this is where we reset the TTL for
-        // tracked blobs. if this TTL reset is skipped, some of the blobs retained by future commits may
-        // be deleted in the background by the blob store, leading to data loss.
-        info("Cleaning up stale state from previous run for taskName: %s" format taskName)
-        commitManager.cleanUp(checkpointV2.getCheckpointId, checkpointV2.getStateCheckpointMarkers)
-      }
+    var checkpoint: Checkpoint = lastTaskCheckpoint
+    if (offsetManager != null && isStandByTask) {

Review Comment:
   Document _why_ special handling for standby.



##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreRestoreManager.java:
##########
@@ -119,13 +119,18 @@ public BlobStoreRestoreManager(TaskModel taskModel, ExecutorService restoreExecu
 
   @Override
   public void init(Checkpoint checkpoint) {
+    init(checkpoint, false);
+  }
+
+  public void init(Checkpoint checkpoint, Boolean getDeletedBlob) {

Review Comment:
   Add javadoc to public method.



##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreRestoreManager.java:
##########
@@ -254,8 +263,10 @@ static CompletableFuture<Void> restoreStores(String jobName, String jobId, TaskN
         throw new SamzaException(String.format("Error deleting store directory: %s", storeDir), e);
       }
 
+      // If getDeletedBlobs is enabled - always restore so that we get all the blobs, including the deleted blobs,
+      // immediately restore it locally and backup to create new checkpoint.
       boolean shouldRestore = shouldRestore(taskName.getTaskName(), storeName, dirIndex,
-          storeCheckpointDir, storageConfig, dirDiffUtil);
+          storeCheckpointDir, storageConfig, dirDiffUtil) || getDeletedBlobs;

Review Comment:
   Move the getDeletedBlob check to beginning to short circuit the rest.
   
   Are you sure this should be short circuiting / overriding all the other checks in shouldRestore? Not obvious to me. Any scenarios where shouldRestore returns false but we want to force restore?



##########
samza-core/src/main/scala/org/apache/samza/container/TaskInstance.scala:
##########
@@ -149,31 +149,33 @@ class TaskInstance(
     }
   }
 
-  def initTask {
+  def initTask(lastTaskCheckpoint: Checkpoint) {

Review Comment:
   See comment above.



##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManagerUtil.java:
##########
@@ -411,4 +438,292 @@ public static Set<String> getSideInputStoreNames(
     }
     return sideInputStores;
   }
+
+  public static List<Future<Void>> initAndRestoreTaskInstances(

Review Comment:
   Add Javadoc. Also document behavior w.r.t. deleted blobs.



##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManagerUtil.java:
##########
@@ -411,4 +438,292 @@ public static Set<String> getSideInputStoreNames(
     }
     return sideInputStores;
   }
+
+  public static List<Future<Void>> initAndRestoreTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, SamzaContainerMetrics samzaContainerMetrics,
+      CheckpointManager checkpointManager, JobContext jobContext, ContainerModel containerModel,
+      Map<TaskName, Checkpoint> taskCheckpoints, Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames,
+      Config config, ExecutorService executor, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      File loggerStoreDir, Map<String, SystemConsumer> storeConsumers) {
+
+    Set<String> forceRestoreTasks = new HashSet<>();
+    // Initialize each TaskStorageManager.
+    taskRestoreManagers.forEach((taskName, restoreManagers) ->
+        restoreManagers.forEach((factoryName, taskRestoreManager) -> {
+          try {
+            taskRestoreManager.init(taskCheckpoints.get(taskName));
+          } catch (SamzaException ex) {
+            if (isUnwrappedExceptionDeletedException(ex) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+              // Get deleted SnapshotIndex blob with GetDeleted and mark the task to be restored with GetDeleted as well.
+              // this ensures that the restore downloads the snapshot, recreates a new snapshot, uploads it to blob store
+              // and creates a new checkpoint.
+              ((BlobStoreRestoreManager) taskRestoreManager).init(taskCheckpoints.get(taskName), true);

Review Comment:
   Can you confirm init is safe to be called twice? Also document that this can happen in BSRM init javadoc.



##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java:
##########
@@ -208,9 +205,9 @@ public ContainerStorageManager(
   }
 
 
-  public void start() throws SamzaException, InterruptedException {
+  public Map<TaskName, Checkpoint> start() throws SamzaException, InterruptedException {

Review Comment:
   Same for later methods.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] shekhars-li commented on a diff in pull request #1676: [SAMZA-2787] GetDeleted API and Recover from DeletedException

Posted by "shekhars-li (via GitHub)" <gi...@apache.org>.
shekhars-li commented on code in PR #1676:
URL: https://github.com/apache/samza/pull/1676#discussion_r1286280436


##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManagerUtil.java:
##########
@@ -411,4 +438,292 @@ public static Set<String> getSideInputStoreNames(
     }
     return sideInputStores;
   }
+
+  public static List<Future<Void>> initAndRestoreTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, SamzaContainerMetrics samzaContainerMetrics,
+      CheckpointManager checkpointManager, JobContext jobContext, ContainerModel containerModel,
+      Map<TaskName, Checkpoint> taskCheckpoints, Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames,
+      Config config, ExecutorService executor, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      File loggerStoreDir, Map<String, SystemConsumer> storeConsumers) {
+
+    Set<String> forceRestoreTasks = new HashSet<>();
+    // Initialize each TaskStorageManager.
+    taskRestoreManagers.forEach((taskName, restoreManagers) ->
+        restoreManagers.forEach((factoryName, taskRestoreManager) -> {
+          try {
+            taskRestoreManager.init(taskCheckpoints.get(taskName));
+          } catch (SamzaException ex) {
+            if (isUnwrappedExceptionDeletedException(ex) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+              // Get deleted SnapshotIndex blob with GetDeleted and mark the task to be restored with GetDeleted as well.
+              // this ensures that the restore downloads the snapshot, recreates a new snapshot, uploads it to blob store
+              // and creates a new checkpoint.
+              ((BlobStoreRestoreManager) taskRestoreManager).init(taskCheckpoints.get(taskName), true);
+              forceRestoreTasks.add(taskName.getTaskName());
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("init failed for task: %s with GetDeleted set to true", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex);
+            }
+          }
+        })
+    );
+
+    // Start each store consumer once.
+    // Note: These consumers are per system and only changelog system store consumers will be started.
+    // Some TaskRestoreManagers may not require the consumer to be started, but due to the agnostic nature of
+    // ContainerStorageManager we always start the changelog consumer here in case it is required
+    storeConsumers.values().stream().distinct().forEach(SystemConsumer::start);
+
+    return restoreAllTaskInstances(taskRestoreManagers, taskCheckpoints, taskBackendFactoryToStoreNames, jobContext,
+        containerModel, samzaContainerMetrics, checkpointManager, config, taskInstanceMetrics, executor, loggerStoreDir,
+        forceRestoreTasks);
+  }
+
+  /**
+   * Restores all TaskInstances and returns a future for each TaskInstance restore. If this restore fails with DeletedException
+   * it will retry the restore with getDeleted flag, get all the blobs, recreate a new checkpoint in blob store and
+   * write that checkpoint to checkpoint topic.
+   * @param taskRestoreManagers map of TaskName to factory name to TaskRestoreManager map.
+   */
+  public static List<Future<Void>> restoreAllTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, Map<TaskName, Checkpoint> taskCheckpoints,
+      Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames, JobContext jobContext,
+      ContainerModel containerModel, SamzaContainerMetrics samzaContainerMetrics, CheckpointManager checkpointManager,
+      Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics, ExecutorService executor,
+      File loggedStoreDir, Set<String> forceRestoreTask) {
+
+    List<Future<Void>> taskRestoreFutures = new ArrayList<>();
+
+    // Submit restore callable for each taskInstance
+    taskRestoreManagers.forEach((taskInstanceName, restoreManagersMap) -> {
+      // Submit for each restore factory
+      restoreManagersMap.forEach((factoryName, taskRestoreManager) -> {
+        long startTime = System.currentTimeMillis();
+        String taskName = taskInstanceName.getTaskName();
+        LOG.info("Starting restore for state for task: {}", taskName);
+
+        CompletableFuture<Void> restoreFuture;
+        if (forceRestoreTask.contains(taskName) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+          restoreFuture = ((BlobStoreRestoreManager) taskRestoreManager).restore(true);
+        } else {
+          restoreFuture = taskRestoreManager.restore();
+        }
+
+        CompletableFuture<Void> taskRestoreFuture = restoreFuture.handle((res, ex) -> {
+          updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+
+          if (ex != null) {
+            if (isUnwrappedExceptionDeletedException(ex)) {
+              LOG.warn(
+                  "Restore state for task: {} received DeletedException. Reattempting with getDeletedBlobs enabled",
+                  taskInstanceName.getTaskName());
+
+              // Try to restore with getDeleted flag
+              CompletableFuture<Void> future =
+                  restoreTaskStoresAndCreateCheckpointWithGetDeleted(taskInstanceName, taskCheckpoints, checkpointManager,
+                      taskRestoreManager, config, taskInstanceMetrics, executor,
+                      taskBackendFactoryToStoreNames.get(taskInstanceName).get(factoryName), loggedStoreDir, jobContext,
+                      containerModel);
+              try {
+                if (future != null) {
+                  future.join(); // Block and restore deleted state before continuing
+                }
+              } catch (Exception e) {
+                String msg = String.format("Unable to recover from DeletedException for task %s.", taskName);
+                throw new SamzaException(msg, e);
+              } finally {
+                updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+              }
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("Error restoring state for task: %s", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex); // wrap in unchecked exception to throw from lambda
+            }
+          }
+
+          // Stop all persistent stores after restoring. Certain persistent stores opened in BulkLoad mode are compacted
+          // on stop, so paralleling stop() also parallelizes their compaction (a time-intensive operation).
+          try {
+            taskRestoreManager.close();
+          } catch (Exception e) {
+            LOG.error("Error closing restore manager for task: {} after {} restore", taskName,
+                ex != null ? "unsuccessful" : "successful", e);
+            // ignore exception from close. container may still be able to continue processing/backups
+            // if restore manager close fails.
+          }
+          return null;
+        });
+        taskRestoreFutures.add(taskRestoreFuture);
+      });
+    });
+
+    return taskRestoreFutures;
+  }
+
+  public static CompletableFuture<Void> restoreTaskStoresAndCreateCheckpointWithGetDeleted(TaskName taskName,
+      Map<TaskName, Checkpoint> taskCheckpoints, CheckpointManager checkpointManager,
+      TaskRestoreManager taskRestoreManager, Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      ExecutorService executor, Set<String> storesToRestore, File loggedStoreBaseDirectory, JobContext jobContext,
+      ContainerModel containerModel) {
+
+    // if taskInstanceMetrics are specified use those for store metrics,
+    // otherwise (in case of StorageRecovery) use a blank MetricsRegistryMap
+    MetricsRegistry metricsRegistry =
+        taskInstanceMetrics.get(taskName) != null ? taskInstanceMetrics.get(taskName).registry()
+            : new MetricsRegistryMap();
+
+    BlobStoreManager blobStoreManager = getBlobStoreManager(config, executor);
+    JobConfig jobConfig = new JobConfig(config);
+    BlobStoreUtil blobStoreUtil =
+        new BlobStoreUtil(blobStoreManager, executor, new BlobStoreConfig(config), null,
+            new BlobStoreRestoreManagerMetrics(metricsRegistry));
+
+    BlobStoreRestoreManager blobStoreRestoreManager = (BlobStoreRestoreManager) taskRestoreManager;
+
+    CheckpointId checkpointId = CheckpointId.create();
+    Map<String, String> oldSCMs = ((CheckpointV2) taskCheckpoints.get(taskName))
+        .getStateCheckpointMarkers().get(BlobStoreStateBackendFactory.class.getName());
+
+    // Returns a single future that guarantees all the following are completed, in this order:
+    // 1. Restore state locally by getting deleted blobs from the blob store.
+    // 2. Create a new Checkpoint from restored state and back it up on the blob store.
+    // 3. Clean up old/deleted Snapshot
+    // 4. Remove TTL from the new Snapshot on the blob store
+    // 5. Write the new checkpoint to checkpoint topic and update taskCheckpoints map
+
+    // 1. restore state with getDeleted flag set to true
+    return blobStoreRestoreManager.restore(true).thenCompose(r -> {
+      // 2. Create and backup new checkpoint on the blob store.
+      CompletableFuture<Map<String, String>> uploadSCMs =
+          createNewCheckpointAndBackupStores(jobContext, containerModel, config, taskName, storesToRestore, checkpointId,
+              loggedStoreBaseDirectory, blobStoreManager, metricsRegistry, executor);
+      CompletableFuture<Void> future = uploadSCMs.thenCompose(scms -> {
+        // 3. Delete prev SnapshotIndex including files/subdirs,
+        Map<String, CompletionStage<Void>> deleteOldSnapshotsFutures =
+            deletePrevSnapshots(blobStoreUtil, oldSCMs, taskName, jobConfig);
+        // 4. Mark new Snapshots to never expire
+        Map<String, CompletionStage<Void>> removeNewSnapshotsTTLFutures =
+            removeTTLNewSnapshots(blobStoreUtil, scms, taskName, jobConfig);
+        // Combined future of delete old snapshots and removeTTL of new snapshots
+        CompletableFuture<Void> removeTTLFuture = CompletableFuture.allOf(FutureUtil.mapToFuture(deleteOldSnapshotsFutures),
+            FutureUtil.mapToFuture(removeNewSnapshotsTTLFutures));
+        // 5. Update taskCheckpoints map with new checkpoint
+        CompletableFuture<Void> checkpointFuture = removeTTLFuture.thenRun(() -> {
+          Checkpoint newCheckpoint = writeNewCheckpoint(taskName, checkpointId, scms, checkpointManager);
+          taskCheckpoints.put(taskName, newCheckpoint);
+        });
+        return checkpointFuture;
+      });
+      return future;
+    }).exceptionally(ex -> {
+      String msg = String.format("Restore for task %s failed with get deleted bob", taskName);
+      throw new SamzaException(msg, ex);
+    });
+  }
+
+  private static CompletableFuture<Map<String, String>> createNewCheckpointAndBackupStores(JobContext jobContext,

Review Comment:
   Update the name to reflect main task -> backup stores.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] shekhars-li commented on a diff in pull request #1676: [SAMZA-2787] GetDeleted API and Recover from DeletedException

Posted by "shekhars-li (via GitHub)" <gi...@apache.org>.
shekhars-li commented on code in PR #1676:
URL: https://github.com/apache/samza/pull/1676#discussion_r1287908574


##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreRestoreManager.java:
##########
@@ -212,9 +213,17 @@ static CompletableFuture<Void> restoreStores(String jobName, String jobId, TaskN
       File loggedBaseDir, StorageConfig storageConfig, BlobStoreRestoreManagerMetrics metrics,
       StorageManagerUtil storageManagerUtil, BlobStoreUtil blobStoreUtil, DirDiffUtil dirDiffUtil,
       ExecutorService executor) {
+    return restoreStores(jobName, jobId, taskName, storesToRestore, prevStoreSnapshotIndexes, loggedBaseDir, storageConfig,
+        metrics, storageManagerUtil, blobStoreUtil, dirDiffUtil, executor, false);
+  }
+
+  public static CompletableFuture<Void> restoreStores(String jobName, String jobId, TaskName taskName, Set<String> storesToRestore,

Review Comment:
   This is because it's called from CSMUtil directly in case the restore fails. This is separate from restore() which is impl of interface TaskRestoreManager. Both versions of restores are required. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] shekhars-li commented on a diff in pull request #1676: [SAMZA-2787] GetDeleted API and Recover from DeletedException

Posted by "shekhars-li (via GitHub)" <gi...@apache.org>.
shekhars-li commented on code in PR #1676:
URL: https://github.com/apache/samza/pull/1676#discussion_r1287952358


##########
samza-core/src/main/scala/org/apache/samza/container/SamzaContainer.scala:
##########
@@ -981,7 +981,7 @@ class SamzaContainer(
     }
   }
 
-  def startStores {
+  def startStores: util.Map[TaskName, Checkpoint] = {

Review Comment:
   I updated them in this commit: https://github.com/apache/samza/pull/1676/commits/8af30747c1ade4aa7bfa5f899658ea75e0d37aea
   
   Should show up now. Not sure why it wasn't visible before. Same for the startTask method description. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] shekhars-li commented on a diff in pull request #1676: [SAMZA-2787] GetDeleted API and Recover from DeletedException

Posted by "shekhars-li (via GitHub)" <gi...@apache.org>.
shekhars-li commented on code in PR #1676:
URL: https://github.com/apache/samza/pull/1676#discussion_r1287913093


##########
samza-core/src/main/java/org/apache/samza/storage/TaskStorageCommitManager.java:
##########
@@ -93,14 +93,18 @@ public TaskStorageCommitManager(TaskName taskName, Map<String, TaskBackupManager
     this.metrics = metrics;
   }
 
-  public void init() {
+  public void init(Checkpoint checkpoint) {
     // Assuming that container storage manager has already started and created to stores
     storageEngines = containerStorageManager.getAllStores(taskName);
     if (checkpointManager != null) {
-      Checkpoint checkpoint = checkpointManager.readLastCheckpoint(taskName);
+      // In case of standby containers

Review Comment:
   Updated this comment in TaskInstance where we are actually passing null/checkpoint depending on standby/active container. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] shekhars-li commented on a diff in pull request #1676: [SAMZA-2787] GetDeleted API and Recover from DeletedException

Posted by "shekhars-li (via GitHub)" <gi...@apache.org>.
shekhars-li commented on code in PR #1676:
URL: https://github.com/apache/samza/pull/1676#discussion_r1286269622


##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/util/BlobStoreUtil.java:
##########
@@ -162,15 +167,7 @@ public Map<String, Pair<String, SnapshotIndex>> getStoreSnapshotIndexes(
     }
 
     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();
+      return FutureUtil.toFutureOfMap(storeSnapshotIndexFutures).join();

Review Comment:
   Previously, we were ignoring the DeletedException for getSnapshotIndex. This was incorrect behavior. Now, if we get an exception, we throw an exception and fail. On restart, we recover by getting the snapshot index with getDeleted, and recreate the checkpoint. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] prateekm commented on a diff in pull request #1676: [SAMZA-2787] GetDeleted API and Recover from DeletedException

Posted by "prateekm (via GitHub)" <gi...@apache.org>.
prateekm commented on code in PR #1676:
URL: https://github.com/apache/samza/pull/1676#discussion_r1288927853


##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManagerUtil.java:
##########
@@ -411,4 +438,292 @@ public static Set<String> getSideInputStoreNames(
     }
     return sideInputStores;
   }
+
+  public static List<Future<Void>> initAndRestoreTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, SamzaContainerMetrics samzaContainerMetrics,
+      CheckpointManager checkpointManager, JobContext jobContext, ContainerModel containerModel,
+      Map<TaskName, Checkpoint> taskCheckpoints, Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames,
+      Config config, ExecutorService executor, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      File loggerStoreDir, Map<String, SystemConsumer> storeConsumers) {
+
+    Set<String> forceRestoreTasks = new HashSet<>();
+    // Initialize each TaskStorageManager.
+    taskRestoreManagers.forEach((taskName, restoreManagers) ->
+        restoreManagers.forEach((factoryName, taskRestoreManager) -> {
+          try {
+            taskRestoreManager.init(taskCheckpoints.get(taskName));
+          } catch (SamzaException ex) {
+            if (isUnwrappedExceptionDeletedException(ex) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+              // Get deleted SnapshotIndex blob with GetDeleted and mark the task to be restored with GetDeleted as well.
+              // this ensures that the restore downloads the snapshot, recreates a new snapshot, uploads it to blob store
+              // and creates a new checkpoint.
+              ((BlobStoreRestoreManager) taskRestoreManager).init(taskCheckpoints.get(taskName), true);
+              forceRestoreTasks.add(taskName.getTaskName());
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("init failed for task: %s with GetDeleted set to true", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex);
+            }
+          }
+        })
+    );
+
+    // Start each store consumer once.
+    // Note: These consumers are per system and only changelog system store consumers will be started.
+    // Some TaskRestoreManagers may not require the consumer to be started, but due to the agnostic nature of
+    // ContainerStorageManager we always start the changelog consumer here in case it is required
+    storeConsumers.values().stream().distinct().forEach(SystemConsumer::start);
+
+    return restoreAllTaskInstances(taskRestoreManagers, taskCheckpoints, taskBackendFactoryToStoreNames, jobContext,
+        containerModel, samzaContainerMetrics, checkpointManager, config, taskInstanceMetrics, executor, loggerStoreDir,
+        forceRestoreTasks);
+  }
+
+  /**
+   * Restores all TaskInstances and returns a future for each TaskInstance restore. If this restore fails with DeletedException
+   * it will retry the restore with getDeleted flag, get all the blobs, recreate a new checkpoint in blob store and
+   * write that checkpoint to checkpoint topic.
+   * @param taskRestoreManagers map of TaskName to factory name to TaskRestoreManager map.
+   */
+  public static List<Future<Void>> restoreAllTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, Map<TaskName, Checkpoint> taskCheckpoints,
+      Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames, JobContext jobContext,
+      ContainerModel containerModel, SamzaContainerMetrics samzaContainerMetrics, CheckpointManager checkpointManager,
+      Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics, ExecutorService executor,
+      File loggedStoreDir, Set<String> forceRestoreTask) {
+
+    List<Future<Void>> taskRestoreFutures = new ArrayList<>();
+
+    // Submit restore callable for each taskInstance
+    taskRestoreManagers.forEach((taskInstanceName, restoreManagersMap) -> {
+      // Submit for each restore factory
+      restoreManagersMap.forEach((factoryName, taskRestoreManager) -> {
+        long startTime = System.currentTimeMillis();
+        String taskName = taskInstanceName.getTaskName();
+        LOG.info("Starting restore for state for task: {}", taskName);
+
+        CompletableFuture<Void> restoreFuture;
+        if (forceRestoreTask.contains(taskName) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+          restoreFuture = ((BlobStoreRestoreManager) taskRestoreManager).restore(true);
+        } else {
+          restoreFuture = taskRestoreManager.restore();
+        }
+
+        CompletableFuture<Void> taskRestoreFuture = restoreFuture.handle((res, ex) -> {
+          updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+
+          if (ex != null) {
+            if (isUnwrappedExceptionDeletedException(ex)) {
+              LOG.warn(
+                  "Restore state for task: {} received DeletedException. Reattempting with getDeletedBlobs enabled",
+                  taskInstanceName.getTaskName());
+
+              // Try to restore with getDeleted flag
+              CompletableFuture<Void> future =
+                  restoreTaskStoresAndCreateCheckpointWithGetDeleted(taskInstanceName, taskCheckpoints, checkpointManager,

Review Comment:
   LGTM, thanks.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] shekhars-li commented on a diff in pull request #1676: [SAMZA-2787] GetDeleted API and Recover from DeletedException

Posted by "shekhars-li (via GitHub)" <gi...@apache.org>.
shekhars-li commented on code in PR #1676:
URL: https://github.com/apache/samza/pull/1676#discussion_r1287896727


##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/util/BlobStoreUtil.java:
##########
@@ -210,6 +216,56 @@ public CompletableFuture<String> putSnapshotIndex(SnapshotIndex snapshotIndex) {
     }, isCauseNonRetriable(), executor, retryPolicyConfig);
   }
 
+  /**
+   * Cleans up a SnapshotIndex by recursively deleting all blobs associated with files/subdirs inside the SnapshotIndex
+   * and finally deletes SnapshotIndex blob itself. This is done by getting the SnapshotIndex first.
+   * @param snapshotIndexBlobId Blob if of SnapshotIndex
+   * @param requestMetadata Metadata of the request
+   */
+  public CompletionStage<Void> cleanSnapshotIndex(String snapshotIndexBlobId, Metadata requestMetadata) {
+    return cleanSnapshotIndex(snapshotIndexBlobId, requestMetadata, false);
+  }
+
+  /**
+   * Cleans up a SnapshotIndex by recursively deleting all blobs associated with files/subdirs inside the SnapshotIndex
+   * and finally deletes SnapshotIndex blob itself. This is done by getting the SnapshotIndex first.
+   * @param snapshotIndexBlobId Blob if of SnapshotIndex
+   * @param requestMetadata Metadata of the request
+   * @param getDeletedBlob Gets SnapshotIndex with getDeleted flag set
+   */
+  public CompletionStage<Void> cleanSnapshotIndex(String snapshotIndexBlobId, Metadata requestMetadata, Boolean getDeletedBlob) {
+    Metadata getSnapshotRequest = new Metadata(Metadata.SNAPSHOT_INDEX_PAYLOAD_PATH, Optional.empty(), requestMetadata.getJobName(),
+        requestMetadata.getJobId(), requestMetadata.getTaskName(), requestMetadata.getStoreName());
+    SnapshotIndex snapshotIndex = getSnapshotIndex(snapshotIndexBlobId, getSnapshotRequest, getDeletedBlob).join();

Review Comment:
   That was left over from a test. Removed it. Thanks for catching this. Updated



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] shekhars-li commented on a diff in pull request #1676: [SAMZA-2787] GetDeleted API and Recover from DeletedException

Posted by "shekhars-li (via GitHub)" <gi...@apache.org>.
shekhars-li commented on code in PR #1676:
URL: https://github.com/apache/samza/pull/1676#discussion_r1287897518


##########
samza-core/src/main/java/org/apache/samza/util/FutureUtil.java:
##########
@@ -139,6 +139,17 @@ public static <K, V> CompletableFuture<Map<K, V>> toFutureOfMap(
     });
   }
 
+  /**
+   * Helper method to convert: {@code Map<K, CompletionStage<Void>>}
+   * to:                       {@code CompletableFuture<Void>}
+   *
+   * Returns a future that completes when all value futures complete.
+   * Returned future completes exceptionally if any of the value futures complete exceptionally.
+   */
+  public static CompletableFuture<Void> mapToFuture(Map<String, CompletionStage<Void>> map) {

Review Comment:
   This variant is not needed anymore. I removed it. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] shekhars-li commented on a diff in pull request #1676: [SAMZA-2787] GetDeleted API and Recover from DeletedException

Posted by "shekhars-li (via GitHub)" <gi...@apache.org>.
shekhars-li commented on code in PR #1676:
URL: https://github.com/apache/samza/pull/1676#discussion_r1289157811


##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java:
##########
@@ -301,79 +303,28 @@ private void restoreStores() throws InterruptedException {
               samzaContainerMetrics, taskInstanceMetrics, taskInstanceCollectors, serdes,
               loggedStoreBaseDirectory, nonLoggedStoreBaseDirectory, config, clock);
       taskRestoreManagers.put(taskName, taskStoreRestoreManagers);
+      taskBackendFactoryToStoreNames.put(taskName, backendFactoryToStoreNames);
     });
 
-    // Initialize each TaskStorageManager
-    taskRestoreManagers.forEach((taskName, restoreManagers) ->
-        restoreManagers.forEach((factoryName, taskRestoreManager) ->
-            taskRestoreManager.init(taskCheckpoints.get(taskName))
-        )
-    );
-
-    // Start each store consumer once.
-    // Note: These consumers are per system and only changelog system store consumers will be started.
-    // Some TaskRestoreManagers may not require the consumer to to be started, but due to the agnostic nature of
-    // ContainerStorageManager we always start the changelog consumer here in case it is required
-    this.storeConsumers.values().stream().distinct().forEach(SystemConsumer::start);
-
-    List<Future<Void>> taskRestoreFutures = new ArrayList<>();
-
-    // Submit restore callable for each taskInstance
-    taskRestoreManagers.forEach((taskInstance, restoreManagersMap) -> {
-      // Submit for each restore factory
-      restoreManagersMap.forEach((factoryName, taskRestoreManager) -> {
-        long startTime = System.currentTimeMillis();
-        String taskName = taskInstance.getTaskName();
-        LOG.info("Starting restore for state for task: {}", taskName);
-        CompletableFuture<Void> restoreFuture = taskRestoreManager.restore().handle((res, ex) -> {
-          // Stop all persistent stores after restoring. Certain persistent stores opened in BulkLoad mode are compacted
-          // on stop, so paralleling stop() also parallelizes their compaction (a time-intensive operation).
-          try {
-            taskRestoreManager.close();
-          } catch (Exception e) {
-            LOG.error("Error closing restore manager for task: {} after {} restore",
-                taskName, ex != null ? "unsuccessful" : "successful", e);
-            // ignore exception from close. container may still be be able to continue processing/backups
-            // if restore manager close fails.
-          }
-
-          long timeToRestore = System.currentTimeMillis() - startTime;
-          if (samzaContainerMetrics != null) {
-            Gauge taskGauge = samzaContainerMetrics.taskStoreRestorationMetrics().getOrDefault(taskInstance, null);
-
-            if (taskGauge != null) {
-              taskGauge.set(timeToRestore);
-            }
-          }
-
-          if (ex != null) {
-            // log and rethrow exception to communicate restore failure
-            String msg = String.format("Error restoring state for task: %s", taskName);
-            LOG.error(msg, ex);
-            throw new SamzaException(msg, ex); // wrap in unchecked exception to throw from lambda
-          } else {
-            return null;
-          }
-        });
-
-        taskRestoreFutures.add(restoreFuture);
-      });
-    });
+    // Init all taskRestores and if successful, restores all the task stores concurrently
+    CompletableFuture<Map<TaskName, Checkpoint>> initRestoreAndNewCheckpointFuture =
+        ContainerStorageManagerUtil.initAndRestoreTaskInstances(taskRestoreManagers, samzaContainerMetrics,
+            checkpointManager, jobContext, containerModel, taskCheckpoints, taskBackendFactoryToStoreNames, config,
+            restoreExecutor, taskInstanceMetrics, loggedStoreBaseDirectory, storeConsumers);
 
-    // Loop-over the future list to wait for each restore to finish, catch any exceptions during restore and throw
-    // as samza exceptions
-    for (Future<Void> future : taskRestoreFutures) {
-      try {
-        future.get();
-      } catch (InterruptedException e) {
-        LOG.warn("Received an interrupt during store restoration. Interrupting the restore executor to exit "
-            + "prematurely without restoring full state.");
-        restoreExecutor.shutdownNow();
-        throw e;
-      } catch (Exception e) {
-        LOG.error("Exception when restoring state.", e);
-        throw new SamzaException("Exception when restoring state.", e);
-      }
+    // Update the task checkpoints map, if it was updated during the restore. Throw an exception if the restore or
+    // creating a new checkpoint (in case of BlobStoreBackendFactory) failed.
+    try {
+      Map<TaskName, Checkpoint> newTaskCheckpoints = initRestoreAndNewCheckpointFuture.get();
+      taskCheckpoints.putAll(newTaskCheckpoints);

Review Comment:
   The returned map only has taskName -> checkpoint if a new checkpoint was created for a task. Some task restores may complete without any errors. That's why I am adding them to the map, not replacing it.  



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] prateekm commented on a diff in pull request #1676: [SAMZA-2787] GetDeleted API and Recover from DeletedException

Posted by "prateekm (via GitHub)" <gi...@apache.org>.
prateekm commented on code in PR #1676:
URL: https://github.com/apache/samza/pull/1676#discussion_r1287652768


##########
samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreRestoreManager.java:
##########
@@ -119,13 +119,18 @@ public BlobStoreRestoreManager(TaskModel taskModel, ExecutorService restoreExecu
 
   @Override
   public void init(Checkpoint checkpoint) {
+    init(checkpoint, false);
+  }
+
+  public void init(Checkpoint checkpoint, Boolean getDeletedBlob) {
     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, storesToRestore);
+    prevStoreSnapshotIndexes =
+        blobStoreUtil.getStoreSnapshotIndexes(jobName, jobId, taskName, checkpoint, storesToRestore, getDeletedBlob);

Review Comment:
   Discussed offline. We use "snapshot index is deleted" as a indicator for "files are deleted" (since that's the last step in the post-commit cleanup). Making this check upfront allows us to keep the shouldRestore optimization below for the happy path. If snapshot index is deleted, then all files need to be restored with getDeletedBlob = true and restored, regardless of whether there is a local checkpoint or not. Resolving this.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] shekhars-li commented on a diff in pull request #1676: [SAMZA-2787] GetDeleted API and Recover from DeletedException

Posted by "shekhars-li (via GitHub)" <gi...@apache.org>.
shekhars-li commented on code in PR #1676:
URL: https://github.com/apache/samza/pull/1676#discussion_r1287911128


##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManagerUtil.java:
##########
@@ -411,4 +438,292 @@ public static Set<String> getSideInputStoreNames(
     }
     return sideInputStores;
   }
+
+  public static List<Future<Void>> initAndRestoreTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, SamzaContainerMetrics samzaContainerMetrics,
+      CheckpointManager checkpointManager, JobContext jobContext, ContainerModel containerModel,
+      Map<TaskName, Checkpoint> taskCheckpoints, Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames,
+      Config config, ExecutorService executor, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      File loggerStoreDir, Map<String, SystemConsumer> storeConsumers) {
+
+    Set<String> forceRestoreTasks = new HashSet<>();
+    // Initialize each TaskStorageManager.
+    taskRestoreManagers.forEach((taskName, restoreManagers) ->
+        restoreManagers.forEach((factoryName, taskRestoreManager) -> {
+          try {
+            taskRestoreManager.init(taskCheckpoints.get(taskName));
+          } catch (SamzaException ex) {
+            if (isUnwrappedExceptionDeletedException(ex) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+              // Get deleted SnapshotIndex blob with GetDeleted and mark the task to be restored with GetDeleted as well.
+              // this ensures that the restore downloads the snapshot, recreates a new snapshot, uploads it to blob store
+              // and creates a new checkpoint.
+              ((BlobStoreRestoreManager) taskRestoreManager).init(taskCheckpoints.get(taskName), true);
+              forceRestoreTasks.add(taskName.getTaskName());
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("init failed for task: %s with GetDeleted set to true", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex);
+            }
+          }
+        })
+    );
+
+    // Start each store consumer once.
+    // Note: These consumers are per system and only changelog system store consumers will be started.
+    // Some TaskRestoreManagers may not require the consumer to be started, but due to the agnostic nature of
+    // ContainerStorageManager we always start the changelog consumer here in case it is required
+    storeConsumers.values().stream().distinct().forEach(SystemConsumer::start);
+
+    return restoreAllTaskInstances(taskRestoreManagers, taskCheckpoints, taskBackendFactoryToStoreNames, jobContext,
+        containerModel, samzaContainerMetrics, checkpointManager, config, taskInstanceMetrics, executor, loggerStoreDir,
+        forceRestoreTasks);
+  }
+
+  /**
+   * Restores all TaskInstances and returns a future for each TaskInstance restore. If this restore fails with DeletedException
+   * it will retry the restore with getDeleted flag, get all the blobs, recreate a new checkpoint in blob store and
+   * write that checkpoint to checkpoint topic.
+   * @param taskRestoreManagers map of TaskName to factory name to TaskRestoreManager map.
+   */
+  public static List<Future<Void>> restoreAllTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, Map<TaskName, Checkpoint> taskCheckpoints,
+      Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames, JobContext jobContext,
+      ContainerModel containerModel, SamzaContainerMetrics samzaContainerMetrics, CheckpointManager checkpointManager,
+      Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics, ExecutorService executor,
+      File loggedStoreDir, Set<String> forceRestoreTask) {
+
+    List<Future<Void>> taskRestoreFutures = new ArrayList<>();
+
+    // Submit restore callable for each taskInstance
+    taskRestoreManagers.forEach((taskInstanceName, restoreManagersMap) -> {
+      // Submit for each restore factory
+      restoreManagersMap.forEach((factoryName, taskRestoreManager) -> {
+        long startTime = System.currentTimeMillis();
+        String taskName = taskInstanceName.getTaskName();
+        LOG.info("Starting restore for state for task: {}", taskName);
+
+        CompletableFuture<Void> restoreFuture;
+        if (forceRestoreTask.contains(taskName) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+          restoreFuture = ((BlobStoreRestoreManager) taskRestoreManager).restore(true);
+        } else {
+          restoreFuture = taskRestoreManager.restore();
+        }
+
+        CompletableFuture<Void> taskRestoreFuture = restoreFuture.handle((res, ex) -> {
+          updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+
+          if (ex != null) {
+            if (isUnwrappedExceptionDeletedException(ex)) {
+              LOG.warn(
+                  "Restore state for task: {} received DeletedException. Reattempting with getDeletedBlobs enabled",
+                  taskInstanceName.getTaskName());
+
+              // Try to restore with getDeleted flag
+              CompletableFuture<Void> future =
+                  restoreTaskStoresAndCreateCheckpointWithGetDeleted(taskInstanceName, taskCheckpoints, checkpointManager,

Review Comment:
   Renamed it to better reflect the intent. Please let me know if that's better or if I should split.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] shekhars-li commented on a diff in pull request #1676: [SAMZA-2787] GetDeleted API and Recover from DeletedException

Posted by "shekhars-li (via GitHub)" <gi...@apache.org>.
shekhars-li commented on code in PR #1676:
URL: https://github.com/apache/samza/pull/1676#discussion_r1286279330


##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManagerUtil.java:
##########
@@ -411,4 +438,292 @@ public static Set<String> getSideInputStoreNames(
     }
     return sideInputStores;
   }
+
+  public static List<Future<Void>> initAndRestoreTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, SamzaContainerMetrics samzaContainerMetrics,
+      CheckpointManager checkpointManager, JobContext jobContext, ContainerModel containerModel,
+      Map<TaskName, Checkpoint> taskCheckpoints, Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames,
+      Config config, ExecutorService executor, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      File loggerStoreDir, Map<String, SystemConsumer> storeConsumers) {
+
+    Set<String> forceRestoreTasks = new HashSet<>();
+    // Initialize each TaskStorageManager.
+    taskRestoreManagers.forEach((taskName, restoreManagers) ->
+        restoreManagers.forEach((factoryName, taskRestoreManager) -> {
+          try {
+            taskRestoreManager.init(taskCheckpoints.get(taskName));
+          } catch (SamzaException ex) {
+            if (isUnwrappedExceptionDeletedException(ex) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+              // Get deleted SnapshotIndex blob with GetDeleted and mark the task to be restored with GetDeleted as well.
+              // this ensures that the restore downloads the snapshot, recreates a new snapshot, uploads it to blob store
+              // and creates a new checkpoint.
+              ((BlobStoreRestoreManager) taskRestoreManager).init(taskCheckpoints.get(taskName), true);
+              forceRestoreTasks.add(taskName.getTaskName());
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("init failed for task: %s with GetDeleted set to true", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex);
+            }
+          }
+        })
+    );
+
+    // Start each store consumer once.
+    // Note: These consumers are per system and only changelog system store consumers will be started.
+    // Some TaskRestoreManagers may not require the consumer to be started, but due to the agnostic nature of
+    // ContainerStorageManager we always start the changelog consumer here in case it is required
+    storeConsumers.values().stream().distinct().forEach(SystemConsumer::start);
+
+    return restoreAllTaskInstances(taskRestoreManagers, taskCheckpoints, taskBackendFactoryToStoreNames, jobContext,
+        containerModel, samzaContainerMetrics, checkpointManager, config, taskInstanceMetrics, executor, loggerStoreDir,
+        forceRestoreTasks);
+  }
+
+  /**
+   * Restores all TaskInstances and returns a future for each TaskInstance restore. If this restore fails with DeletedException
+   * it will retry the restore with getDeleted flag, get all the blobs, recreate a new checkpoint in blob store and
+   * write that checkpoint to checkpoint topic.
+   * @param taskRestoreManagers map of TaskName to factory name to TaskRestoreManager map.
+   */
+  public static List<Future<Void>> restoreAllTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, Map<TaskName, Checkpoint> taskCheckpoints,
+      Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames, JobContext jobContext,
+      ContainerModel containerModel, SamzaContainerMetrics samzaContainerMetrics, CheckpointManager checkpointManager,
+      Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics, ExecutorService executor,
+      File loggedStoreDir, Set<String> forceRestoreTask) {
+
+    List<Future<Void>> taskRestoreFutures = new ArrayList<>();
+
+    // Submit restore callable for each taskInstance
+    taskRestoreManagers.forEach((taskInstanceName, restoreManagersMap) -> {
+      // Submit for each restore factory
+      restoreManagersMap.forEach((factoryName, taskRestoreManager) -> {
+        long startTime = System.currentTimeMillis();
+        String taskName = taskInstanceName.getTaskName();
+        LOG.info("Starting restore for state for task: {}", taskName);
+
+        CompletableFuture<Void> restoreFuture;
+        if (forceRestoreTask.contains(taskName) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+          restoreFuture = ((BlobStoreRestoreManager) taskRestoreManager).restore(true);
+        } else {
+          restoreFuture = taskRestoreManager.restore();
+        }
+
+        CompletableFuture<Void> taskRestoreFuture = restoreFuture.handle((res, ex) -> {
+          updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+
+          if (ex != null) {
+            if (isUnwrappedExceptionDeletedException(ex)) {
+              LOG.warn(
+                  "Restore state for task: {} received DeletedException. Reattempting with getDeletedBlobs enabled",
+                  taskInstanceName.getTaskName());
+
+              // Try to restore with getDeleted flag
+              CompletableFuture<Void> future =
+                  restoreTaskStoresAndCreateCheckpointWithGetDeleted(taskInstanceName, taskCheckpoints, checkpointManager,
+                      taskRestoreManager, config, taskInstanceMetrics, executor,
+                      taskBackendFactoryToStoreNames.get(taskInstanceName).get(factoryName), loggedStoreDir, jobContext,
+                      containerModel);
+              try {
+                if (future != null) {
+                  future.join(); // Block and restore deleted state before continuing
+                }
+              } catch (Exception e) {
+                String msg = String.format("Unable to recover from DeletedException for task %s.", taskName);
+                throw new SamzaException(msg, e);
+              } finally {
+                updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+              }
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("Error restoring state for task: %s", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex); // wrap in unchecked exception to throw from lambda
+            }
+          }
+
+          // Stop all persistent stores after restoring. Certain persistent stores opened in BulkLoad mode are compacted
+          // on stop, so paralleling stop() also parallelizes their compaction (a time-intensive operation).
+          try {
+            taskRestoreManager.close();
+          } catch (Exception e) {
+            LOG.error("Error closing restore manager for task: {} after {} restore", taskName,
+                ex != null ? "unsuccessful" : "successful", e);
+            // ignore exception from close. container may still be able to continue processing/backups
+            // if restore manager close fails.
+          }
+          return null;
+        });
+        taskRestoreFutures.add(taskRestoreFuture);
+      });
+    });
+
+    return taskRestoreFutures;
+  }
+
+  public static CompletableFuture<Void> restoreTaskStoresAndCreateCheckpointWithGetDeleted(TaskName taskName,
+      Map<TaskName, Checkpoint> taskCheckpoints, CheckpointManager checkpointManager,
+      TaskRestoreManager taskRestoreManager, Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      ExecutorService executor, Set<String> storesToRestore, File loggedStoreBaseDirectory, JobContext jobContext,
+      ContainerModel containerModel) {
+
+    // if taskInstanceMetrics are specified use those for store metrics,
+    // otherwise (in case of StorageRecovery) use a blank MetricsRegistryMap
+    MetricsRegistry metricsRegistry =
+        taskInstanceMetrics.get(taskName) != null ? taskInstanceMetrics.get(taskName).registry()
+            : new MetricsRegistryMap();
+
+    BlobStoreManager blobStoreManager = getBlobStoreManager(config, executor);
+    JobConfig jobConfig = new JobConfig(config);
+    BlobStoreUtil blobStoreUtil =
+        new BlobStoreUtil(blobStoreManager, executor, new BlobStoreConfig(config), null,
+            new BlobStoreRestoreManagerMetrics(metricsRegistry));
+
+    BlobStoreRestoreManager blobStoreRestoreManager = (BlobStoreRestoreManager) taskRestoreManager;
+
+    CheckpointId checkpointId = CheckpointId.create();
+    Map<String, String> oldSCMs = ((CheckpointV2) taskCheckpoints.get(taskName))
+        .getStateCheckpointMarkers().get(BlobStoreStateBackendFactory.class.getName());
+
+    // Returns a single future that guarantees all the following are completed, in this order:
+    // 1. Restore state locally by getting deleted blobs from the blob store.
+    // 2. Create a new Checkpoint from restored state and back it up on the blob store.
+    // 3. Clean up old/deleted Snapshot
+    // 4. Remove TTL from the new Snapshot on the blob store
+    // 5. Write the new checkpoint to checkpoint topic and update taskCheckpoints map
+
+    // 1. restore state with getDeleted flag set to true
+    return blobStoreRestoreManager.restore(true).thenCompose(r -> {
+      // 2. Create and backup new checkpoint on the blob store.
+      CompletableFuture<Map<String, String>> uploadSCMs =
+          createNewCheckpointAndBackupStores(jobContext, containerModel, config, taskName, storesToRestore, checkpointId,
+              loggedStoreBaseDirectory, blobStoreManager, metricsRegistry, executor);
+      CompletableFuture<Void> future = uploadSCMs.thenCompose(scms -> {
+        // 3. Delete prev SnapshotIndex including files/subdirs,
+        Map<String, CompletionStage<Void>> deleteOldSnapshotsFutures =
+            deletePrevSnapshots(blobStoreUtil, oldSCMs, taskName, jobConfig);
+        // 4. Mark new Snapshots to never expire
+        Map<String, CompletionStage<Void>> removeNewSnapshotsTTLFutures =
+            removeTTLNewSnapshots(blobStoreUtil, scms, taskName, jobConfig);
+        // Combined future of delete old snapshots and removeTTL of new snapshots
+        CompletableFuture<Void> removeTTLFuture = CompletableFuture.allOf(FutureUtil.mapToFuture(deleteOldSnapshotsFutures),
+            FutureUtil.mapToFuture(removeNewSnapshotsTTLFutures));
+        // 5. Update taskCheckpoints map with new checkpoint
+        CompletableFuture<Void> checkpointFuture = removeTTLFuture.thenRun(() -> {
+          Checkpoint newCheckpoint = writeNewCheckpoint(taskName, checkpointId, scms, checkpointManager);
+          taskCheckpoints.put(taskName, newCheckpoint);
+        });
+        return checkpointFuture;
+      });
+      return future;
+    }).exceptionally(ex -> {
+      String msg = String.format("Restore for task %s failed with get deleted bob", taskName);
+      throw new SamzaException(msg, ex);
+    });
+  }
+
+  private static CompletableFuture<Map<String, String>> createNewCheckpointAndBackupStores(JobContext jobContext,
+      ContainerModel containerModel, Config config, TaskName taskName, Set<String> storesToBackup,
+      CheckpointId newCheckpointId, File loggedStoreBaseDirectory, BlobStoreManager blobStoreManager,
+      MetricsRegistry metricsRegistry, ExecutorService executor) {
+
+    BlobStoreBackupManagerMetrics blobStoreBackupManagerMetrics = new BlobStoreBackupManagerMetrics(metricsRegistry);
+    BlobStoreBackupManager blobStoreBackupManager =
+        new BlobStoreBackupManager(jobContext.getJobModel(), containerModel, containerModel.getTasks().get(taskName),
+            executor, blobStoreBackupManagerMetrics, config, SystemClock.instance(), loggedStoreBaseDirectory,
+            new StorageManagerUtil(), blobStoreManager);
+
+    // create checkpoint dir as a copy of store dir
+    createCheckpointDirFromStoreDirCopy(taskName, containerModel.getTasks().get(taskName),
+        loggedStoreBaseDirectory, storesToBackup, newCheckpointId);
+    // upload to blob store and return future
+    return blobStoreBackupManager.upload(newCheckpointId, new HashMap<>());
+  }
+
+  private static Checkpoint writeNewCheckpoint(TaskName taskName, CheckpointId checkpointId,
+      Map<String, String> newSCMs, CheckpointManager checkpointManager) {
+    CheckpointV2 oldCheckpoint = (CheckpointV2) checkpointManager.readLastCheckpoint(taskName);
+    Map<SystemStreamPartition, String> checkpointOffsets = oldCheckpoint.getOffsets();
+
+    Map<String, Map<String, String>> oldStateCheckpointMarkers = oldCheckpoint.getStateCheckpointMarkers();
+
+    Map<String, Map<String, String>> newStateCheckpointMarkers = ImmutableMap.<String, Map<String, String>>builder()
+        .put(KafkaChangelogStateBackendFactory.class.getName(), oldStateCheckpointMarkers.get(KafkaChangelogStateBackendFactory.class.getName()))
+        .put(BlobStoreStateBackendFactory.class.getName(), newSCMs)
+        .build();
+
+    CheckpointV2 checkpointV2 = new CheckpointV2(checkpointId, checkpointOffsets, newStateCheckpointMarkers);
+    checkpointManager.writeCheckpoint(taskName, checkpointV2);
+    return checkpointV2;
+  }
+
+  private static void createCheckpointDirFromStoreDirCopy(TaskName taskName, TaskModel taskModel,
+      File loggedStoreBaseDir, Set<String> storeName, CheckpointId checkpointId) {
+    StorageManagerUtil storageManagerUtil = new StorageManagerUtil();
+    for (String store : storeName) {
+      try {
+        File storeDirectory =
+            storageManagerUtil.getTaskStoreDir(loggedStoreBaseDir, store, taskName, taskModel.getTaskMode());
+        File checkpointDir = new File(storageManagerUtil.getStoreCheckpointDir(storeDirectory, checkpointId));
+        FileUtils.copyDirectory(storeDirectory, checkpointDir);
+      } catch (IOException exception) {
+        String msg = String.format("Unable to create a copy of store directory %s into checkpoint dir %s while "
+            + "attempting to recover from DeletedException", store, checkpointId);
+        throw new SamzaException(msg, exception);
+      }
+    }
+  }
+
+  private static BlobStoreManager getBlobStoreManager(Config config, ExecutorService executor) {
+    BlobStoreConfig blobStoreConfig = new BlobStoreConfig(config);
+    String blobStoreManagerFactory = blobStoreConfig.getBlobStoreManagerFactory();
+    BlobStoreManagerFactory factory = ReflectionUtil.getObj(blobStoreManagerFactory, BlobStoreManagerFactory.class);
+    return factory.getRestoreBlobStoreManager(config, executor);
+  }
+
+  private static void updateRestoreTime(long startTime, SamzaContainerMetrics samzaContainerMetrics,
+      TaskName taskInstance) {
+    long timeToRestore = System.currentTimeMillis() - startTime;
+    if (samzaContainerMetrics != null) {
+      Gauge taskGauge = samzaContainerMetrics.taskStoreRestorationMetrics().getOrDefault(taskInstance, null);
+
+      if (taskGauge != null) {
+        taskGauge.set(timeToRestore);
+      }
+    }
+  }
+
+  private static Boolean isUnwrappedExceptionDeletedException(Throwable ex) {
+    Throwable unwrappedException = FutureUtil.unwrapExceptions(CompletionException.class,
+        FutureUtil.unwrapExceptions(SamzaException.class, ex));
+    return unwrappedException instanceof DeletedException;
+  }
+
+  private static Metadata createSnapshotMetadataRequest(TaskName taskName, JobConfig jobConfig, String store) {

Review Comment:
   This creates a request metadata. Updated the name to clarify.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] shekhars-li commented on a diff in pull request #1676: [SAMZA-2787] GetDeleted API and Recover from DeletedException

Posted by "shekhars-li (via GitHub)" <gi...@apache.org>.
shekhars-li commented on code in PR #1676:
URL: https://github.com/apache/samza/pull/1676#discussion_r1286280141


##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManagerUtil.java:
##########
@@ -411,4 +438,292 @@ public static Set<String> getSideInputStoreNames(
     }
     return sideInputStores;
   }
+
+  public static List<Future<Void>> initAndRestoreTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, SamzaContainerMetrics samzaContainerMetrics,
+      CheckpointManager checkpointManager, JobContext jobContext, ContainerModel containerModel,
+      Map<TaskName, Checkpoint> taskCheckpoints, Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames,
+      Config config, ExecutorService executor, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      File loggerStoreDir, Map<String, SystemConsumer> storeConsumers) {
+
+    Set<String> forceRestoreTasks = new HashSet<>();
+    // Initialize each TaskStorageManager.
+    taskRestoreManagers.forEach((taskName, restoreManagers) ->
+        restoreManagers.forEach((factoryName, taskRestoreManager) -> {
+          try {
+            taskRestoreManager.init(taskCheckpoints.get(taskName));
+          } catch (SamzaException ex) {
+            if (isUnwrappedExceptionDeletedException(ex) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+              // Get deleted SnapshotIndex blob with GetDeleted and mark the task to be restored with GetDeleted as well.
+              // this ensures that the restore downloads the snapshot, recreates a new snapshot, uploads it to blob store
+              // and creates a new checkpoint.
+              ((BlobStoreRestoreManager) taskRestoreManager).init(taskCheckpoints.get(taskName), true);
+              forceRestoreTasks.add(taskName.getTaskName());
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("init failed for task: %s with GetDeleted set to true", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex);
+            }
+          }
+        })
+    );
+
+    // Start each store consumer once.
+    // Note: These consumers are per system and only changelog system store consumers will be started.
+    // Some TaskRestoreManagers may not require the consumer to be started, but due to the agnostic nature of
+    // ContainerStorageManager we always start the changelog consumer here in case it is required
+    storeConsumers.values().stream().distinct().forEach(SystemConsumer::start);
+
+    return restoreAllTaskInstances(taskRestoreManagers, taskCheckpoints, taskBackendFactoryToStoreNames, jobContext,
+        containerModel, samzaContainerMetrics, checkpointManager, config, taskInstanceMetrics, executor, loggerStoreDir,
+        forceRestoreTasks);
+  }
+
+  /**
+   * Restores all TaskInstances and returns a future for each TaskInstance restore. If this restore fails with DeletedException
+   * it will retry the restore with getDeleted flag, get all the blobs, recreate a new checkpoint in blob store and
+   * write that checkpoint to checkpoint topic.
+   * @param taskRestoreManagers map of TaskName to factory name to TaskRestoreManager map.
+   */
+  public static List<Future<Void>> restoreAllTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, Map<TaskName, Checkpoint> taskCheckpoints,
+      Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames, JobContext jobContext,
+      ContainerModel containerModel, SamzaContainerMetrics samzaContainerMetrics, CheckpointManager checkpointManager,
+      Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics, ExecutorService executor,
+      File loggedStoreDir, Set<String> forceRestoreTask) {
+
+    List<Future<Void>> taskRestoreFutures = new ArrayList<>();
+
+    // Submit restore callable for each taskInstance
+    taskRestoreManagers.forEach((taskInstanceName, restoreManagersMap) -> {
+      // Submit for each restore factory
+      restoreManagersMap.forEach((factoryName, taskRestoreManager) -> {
+        long startTime = System.currentTimeMillis();
+        String taskName = taskInstanceName.getTaskName();
+        LOG.info("Starting restore for state for task: {}", taskName);
+
+        CompletableFuture<Void> restoreFuture;
+        if (forceRestoreTask.contains(taskName) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+          restoreFuture = ((BlobStoreRestoreManager) taskRestoreManager).restore(true);
+        } else {
+          restoreFuture = taskRestoreManager.restore();
+        }
+
+        CompletableFuture<Void> taskRestoreFuture = restoreFuture.handle((res, ex) -> {
+          updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+
+          if (ex != null) {
+            if (isUnwrappedExceptionDeletedException(ex)) {
+              LOG.warn(
+                  "Restore state for task: {} received DeletedException. Reattempting with getDeletedBlobs enabled",
+                  taskInstanceName.getTaskName());
+
+              // Try to restore with getDeleted flag
+              CompletableFuture<Void> future =
+                  restoreTaskStoresAndCreateCheckpointWithGetDeleted(taskInstanceName, taskCheckpoints, checkpointManager,
+                      taskRestoreManager, config, taskInstanceMetrics, executor,
+                      taskBackendFactoryToStoreNames.get(taskInstanceName).get(factoryName), loggedStoreDir, jobContext,
+                      containerModel);
+              try {
+                if (future != null) {
+                  future.join(); // Block and restore deleted state before continuing
+                }
+              } catch (Exception e) {
+                String msg = String.format("Unable to recover from DeletedException for task %s.", taskName);
+                throw new SamzaException(msg, e);
+              } finally {
+                updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+              }
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("Error restoring state for task: %s", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex); // wrap in unchecked exception to throw from lambda
+            }
+          }
+
+          // Stop all persistent stores after restoring. Certain persistent stores opened in BulkLoad mode are compacted
+          // on stop, so paralleling stop() also parallelizes their compaction (a time-intensive operation).
+          try {
+            taskRestoreManager.close();
+          } catch (Exception e) {
+            LOG.error("Error closing restore manager for task: {} after {} restore", taskName,
+                ex != null ? "unsuccessful" : "successful", e);
+            // ignore exception from close. container may still be able to continue processing/backups
+            // if restore manager close fails.
+          }
+          return null;
+        });
+        taskRestoreFutures.add(taskRestoreFuture);
+      });
+    });
+
+    return taskRestoreFutures;
+  }
+
+  public static CompletableFuture<Void> restoreTaskStoresAndCreateCheckpointWithGetDeleted(TaskName taskName,
+      Map<TaskName, Checkpoint> taskCheckpoints, CheckpointManager checkpointManager,
+      TaskRestoreManager taskRestoreManager, Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      ExecutorService executor, Set<String> storesToRestore, File loggedStoreBaseDirectory, JobContext jobContext,
+      ContainerModel containerModel) {
+
+    // if taskInstanceMetrics are specified use those for store metrics,
+    // otherwise (in case of StorageRecovery) use a blank MetricsRegistryMap
+    MetricsRegistry metricsRegistry =
+        taskInstanceMetrics.get(taskName) != null ? taskInstanceMetrics.get(taskName).registry()
+            : new MetricsRegistryMap();
+
+    BlobStoreManager blobStoreManager = getBlobStoreManager(config, executor);
+    JobConfig jobConfig = new JobConfig(config);
+    BlobStoreUtil blobStoreUtil =
+        new BlobStoreUtil(blobStoreManager, executor, new BlobStoreConfig(config), null,
+            new BlobStoreRestoreManagerMetrics(metricsRegistry));
+
+    BlobStoreRestoreManager blobStoreRestoreManager = (BlobStoreRestoreManager) taskRestoreManager;
+
+    CheckpointId checkpointId = CheckpointId.create();
+    Map<String, String> oldSCMs = ((CheckpointV2) taskCheckpoints.get(taskName))
+        .getStateCheckpointMarkers().get(BlobStoreStateBackendFactory.class.getName());
+
+    // Returns a single future that guarantees all the following are completed, in this order:
+    // 1. Restore state locally by getting deleted blobs from the blob store.
+    // 2. Create a new Checkpoint from restored state and back it up on the blob store.
+    // 3. Clean up old/deleted Snapshot
+    // 4. Remove TTL from the new Snapshot on the blob store
+    // 5. Write the new checkpoint to checkpoint topic and update taskCheckpoints map
+
+    // 1. restore state with getDeleted flag set to true
+    return blobStoreRestoreManager.restore(true).thenCompose(r -> {
+      // 2. Create and backup new checkpoint on the blob store.
+      CompletableFuture<Map<String, String>> uploadSCMs =
+          createNewCheckpointAndBackupStores(jobContext, containerModel, config, taskName, storesToRestore, checkpointId,
+              loggedStoreBaseDirectory, blobStoreManager, metricsRegistry, executor);
+      CompletableFuture<Void> future = uploadSCMs.thenCompose(scms -> {
+        // 3. Delete prev SnapshotIndex including files/subdirs,
+        Map<String, CompletionStage<Void>> deleteOldSnapshotsFutures =
+            deletePrevSnapshots(blobStoreUtil, oldSCMs, taskName, jobConfig);
+        // 4. Mark new Snapshots to never expire
+        Map<String, CompletionStage<Void>> removeNewSnapshotsTTLFutures =
+            removeTTLNewSnapshots(blobStoreUtil, scms, taskName, jobConfig);
+        // Combined future of delete old snapshots and removeTTL of new snapshots
+        CompletableFuture<Void> removeTTLFuture = CompletableFuture.allOf(FutureUtil.mapToFuture(deleteOldSnapshotsFutures),
+            FutureUtil.mapToFuture(removeNewSnapshotsTTLFutures));
+        // 5. Update taskCheckpoints map with new checkpoint
+        CompletableFuture<Void> checkpointFuture = removeTTLFuture.thenRun(() -> {
+          Checkpoint newCheckpoint = writeNewCheckpoint(taskName, checkpointId, scms, checkpointManager);
+          taskCheckpoints.put(taskName, newCheckpoint);
+        });
+        return checkpointFuture;
+      });
+      return future;
+    }).exceptionally(ex -> {
+      String msg = String.format("Restore for task %s failed with get deleted bob", taskName);
+      throw new SamzaException(msg, ex);
+    });
+  }
+
+  private static CompletableFuture<Map<String, String>> createNewCheckpointAndBackupStores(JobContext jobContext,
+      ContainerModel containerModel, Config config, TaskName taskName, Set<String> storesToBackup,
+      CheckpointId newCheckpointId, File loggedStoreBaseDirectory, BlobStoreManager blobStoreManager,
+      MetricsRegistry metricsRegistry, ExecutorService executor) {
+
+    BlobStoreBackupManagerMetrics blobStoreBackupManagerMetrics = new BlobStoreBackupManagerMetrics(metricsRegistry);
+    BlobStoreBackupManager blobStoreBackupManager =
+        new BlobStoreBackupManager(jobContext.getJobModel(), containerModel, containerModel.getTasks().get(taskName),
+            executor, blobStoreBackupManagerMetrics, config, SystemClock.instance(), loggedStoreBaseDirectory,
+            new StorageManagerUtil(), blobStoreManager);
+
+    // create checkpoint dir as a copy of store dir
+    createCheckpointDirFromStoreDirCopy(taskName, containerModel.getTasks().get(taskName),
+        loggedStoreBaseDirectory, storesToBackup, newCheckpointId);
+    // upload to blob store and return future
+    return blobStoreBackupManager.upload(newCheckpointId, new HashMap<>());
+  }
+
+  private static Checkpoint writeNewCheckpoint(TaskName taskName, CheckpointId checkpointId,
+      Map<String, String> newSCMs, CheckpointManager checkpointManager) {
+    CheckpointV2 oldCheckpoint = (CheckpointV2) checkpointManager.readLastCheckpoint(taskName);
+    Map<SystemStreamPartition, String> checkpointOffsets = oldCheckpoint.getOffsets();
+
+    Map<String, Map<String, String>> oldStateCheckpointMarkers = oldCheckpoint.getStateCheckpointMarkers();
+
+    Map<String, Map<String, String>> newStateCheckpointMarkers = ImmutableMap.<String, Map<String, String>>builder()
+        .put(KafkaChangelogStateBackendFactory.class.getName(), oldStateCheckpointMarkers.get(KafkaChangelogStateBackendFactory.class.getName()))

Review Comment:
   Updated a check here. Thanks



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] shekhars-li commented on a diff in pull request #1676: [SAMZA-2787] GetDeleted API and Recover from DeletedException

Posted by "shekhars-li (via GitHub)" <gi...@apache.org>.
shekhars-li commented on code in PR #1676:
URL: https://github.com/apache/samza/pull/1676#discussion_r1286278769


##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManagerUtil.java:
##########
@@ -411,4 +438,292 @@ public static Set<String> getSideInputStoreNames(
     }
     return sideInputStores;
   }
+
+  public static List<Future<Void>> initAndRestoreTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, SamzaContainerMetrics samzaContainerMetrics,
+      CheckpointManager checkpointManager, JobContext jobContext, ContainerModel containerModel,
+      Map<TaskName, Checkpoint> taskCheckpoints, Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames,
+      Config config, ExecutorService executor, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      File loggerStoreDir, Map<String, SystemConsumer> storeConsumers) {
+
+    Set<String> forceRestoreTasks = new HashSet<>();
+    // Initialize each TaskStorageManager.
+    taskRestoreManagers.forEach((taskName, restoreManagers) ->
+        restoreManagers.forEach((factoryName, taskRestoreManager) -> {
+          try {
+            taskRestoreManager.init(taskCheckpoints.get(taskName));
+          } catch (SamzaException ex) {
+            if (isUnwrappedExceptionDeletedException(ex) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+              // Get deleted SnapshotIndex blob with GetDeleted and mark the task to be restored with GetDeleted as well.
+              // this ensures that the restore downloads the snapshot, recreates a new snapshot, uploads it to blob store
+              // and creates a new checkpoint.
+              ((BlobStoreRestoreManager) taskRestoreManager).init(taskCheckpoints.get(taskName), true);
+              forceRestoreTasks.add(taskName.getTaskName());
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("init failed for task: %s with GetDeleted set to true", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex);
+            }
+          }
+        })
+    );
+
+    // Start each store consumer once.
+    // Note: These consumers are per system and only changelog system store consumers will be started.
+    // Some TaskRestoreManagers may not require the consumer to be started, but due to the agnostic nature of
+    // ContainerStorageManager we always start the changelog consumer here in case it is required
+    storeConsumers.values().stream().distinct().forEach(SystemConsumer::start);
+
+    return restoreAllTaskInstances(taskRestoreManagers, taskCheckpoints, taskBackendFactoryToStoreNames, jobContext,
+        containerModel, samzaContainerMetrics, checkpointManager, config, taskInstanceMetrics, executor, loggerStoreDir,
+        forceRestoreTasks);
+  }
+
+  /**
+   * Restores all TaskInstances and returns a future for each TaskInstance restore. If this restore fails with DeletedException
+   * it will retry the restore with getDeleted flag, get all the blobs, recreate a new checkpoint in blob store and
+   * write that checkpoint to checkpoint topic.
+   * @param taskRestoreManagers map of TaskName to factory name to TaskRestoreManager map.
+   */
+  public static List<Future<Void>> restoreAllTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, Map<TaskName, Checkpoint> taskCheckpoints,
+      Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames, JobContext jobContext,
+      ContainerModel containerModel, SamzaContainerMetrics samzaContainerMetrics, CheckpointManager checkpointManager,
+      Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics, ExecutorService executor,
+      File loggedStoreDir, Set<String> forceRestoreTask) {
+
+    List<Future<Void>> taskRestoreFutures = new ArrayList<>();
+
+    // Submit restore callable for each taskInstance
+    taskRestoreManagers.forEach((taskInstanceName, restoreManagersMap) -> {
+      // Submit for each restore factory
+      restoreManagersMap.forEach((factoryName, taskRestoreManager) -> {
+        long startTime = System.currentTimeMillis();
+        String taskName = taskInstanceName.getTaskName();
+        LOG.info("Starting restore for state for task: {}", taskName);
+
+        CompletableFuture<Void> restoreFuture;
+        if (forceRestoreTask.contains(taskName) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+          restoreFuture = ((BlobStoreRestoreManager) taskRestoreManager).restore(true);
+        } else {
+          restoreFuture = taskRestoreManager.restore();
+        }
+
+        CompletableFuture<Void> taskRestoreFuture = restoreFuture.handle((res, ex) -> {
+          updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+
+          if (ex != null) {
+            if (isUnwrappedExceptionDeletedException(ex)) {
+              LOG.warn(
+                  "Restore state for task: {} received DeletedException. Reattempting with getDeletedBlobs enabled",
+                  taskInstanceName.getTaskName());
+
+              // Try to restore with getDeleted flag
+              CompletableFuture<Void> future =
+                  restoreTaskStoresAndCreateCheckpointWithGetDeleted(taskInstanceName, taskCheckpoints, checkpointManager,
+                      taskRestoreManager, config, taskInstanceMetrics, executor,
+                      taskBackendFactoryToStoreNames.get(taskInstanceName).get(factoryName), loggedStoreDir, jobContext,
+                      containerModel);
+              try {
+                if (future != null) {
+                  future.join(); // Block and restore deleted state before continuing
+                }
+              } catch (Exception e) {
+                String msg = String.format("Unable to recover from DeletedException for task %s.", taskName);
+                throw new SamzaException(msg, e);
+              } finally {
+                updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+              }
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("Error restoring state for task: %s", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex); // wrap in unchecked exception to throw from lambda
+            }
+          }
+
+          // Stop all persistent stores after restoring. Certain persistent stores opened in BulkLoad mode are compacted
+          // on stop, so paralleling stop() also parallelizes their compaction (a time-intensive operation).
+          try {
+            taskRestoreManager.close();
+          } catch (Exception e) {
+            LOG.error("Error closing restore manager for task: {} after {} restore", taskName,
+                ex != null ? "unsuccessful" : "successful", e);
+            // ignore exception from close. container may still be able to continue processing/backups
+            // if restore manager close fails.
+          }
+          return null;
+        });
+        taskRestoreFutures.add(taskRestoreFuture);
+      });
+    });
+
+    return taskRestoreFutures;
+  }
+
+  public static CompletableFuture<Void> restoreTaskStoresAndCreateCheckpointWithGetDeleted(TaskName taskName,
+      Map<TaskName, Checkpoint> taskCheckpoints, CheckpointManager checkpointManager,
+      TaskRestoreManager taskRestoreManager, Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      ExecutorService executor, Set<String> storesToRestore, File loggedStoreBaseDirectory, JobContext jobContext,
+      ContainerModel containerModel) {
+
+    // if taskInstanceMetrics are specified use those for store metrics,
+    // otherwise (in case of StorageRecovery) use a blank MetricsRegistryMap
+    MetricsRegistry metricsRegistry =
+        taskInstanceMetrics.get(taskName) != null ? taskInstanceMetrics.get(taskName).registry()
+            : new MetricsRegistryMap();
+
+    BlobStoreManager blobStoreManager = getBlobStoreManager(config, executor);
+    JobConfig jobConfig = new JobConfig(config);
+    BlobStoreUtil blobStoreUtil =
+        new BlobStoreUtil(blobStoreManager, executor, new BlobStoreConfig(config), null,
+            new BlobStoreRestoreManagerMetrics(metricsRegistry));
+
+    BlobStoreRestoreManager blobStoreRestoreManager = (BlobStoreRestoreManager) taskRestoreManager;
+
+    CheckpointId checkpointId = CheckpointId.create();
+    Map<String, String> oldSCMs = ((CheckpointV2) taskCheckpoints.get(taskName))
+        .getStateCheckpointMarkers().get(BlobStoreStateBackendFactory.class.getName());
+
+    // Returns a single future that guarantees all the following are completed, in this order:
+    // 1. Restore state locally by getting deleted blobs from the blob store.
+    // 2. Create a new Checkpoint from restored state and back it up on the blob store.
+    // 3. Clean up old/deleted Snapshot
+    // 4. Remove TTL from the new Snapshot on the blob store
+    // 5. Write the new checkpoint to checkpoint topic and update taskCheckpoints map
+
+    // 1. restore state with getDeleted flag set to true
+    return blobStoreRestoreManager.restore(true).thenCompose(r -> {
+      // 2. Create and backup new checkpoint on the blob store.
+      CompletableFuture<Map<String, String>> uploadSCMs =
+          createNewCheckpointAndBackupStores(jobContext, containerModel, config, taskName, storesToRestore, checkpointId,
+              loggedStoreBaseDirectory, blobStoreManager, metricsRegistry, executor);
+      CompletableFuture<Void> future = uploadSCMs.thenCompose(scms -> {
+        // 3. Delete prev SnapshotIndex including files/subdirs,
+        Map<String, CompletionStage<Void>> deleteOldSnapshotsFutures =
+            deletePrevSnapshots(blobStoreUtil, oldSCMs, taskName, jobConfig);
+        // 4. Mark new Snapshots to never expire
+        Map<String, CompletionStage<Void>> removeNewSnapshotsTTLFutures =
+            removeTTLNewSnapshots(blobStoreUtil, scms, taskName, jobConfig);
+        // Combined future of delete old snapshots and removeTTL of new snapshots
+        CompletableFuture<Void> removeTTLFuture = CompletableFuture.allOf(FutureUtil.mapToFuture(deleteOldSnapshotsFutures),

Review Comment:
   Updated to chain them. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] shekhars-li commented on a diff in pull request #1676: [SAMZA-2787] GetDeleted API and Recover from DeletedException

Posted by "shekhars-li (via GitHub)" <gi...@apache.org>.
shekhars-li commented on code in PR #1676:
URL: https://github.com/apache/samza/pull/1676#discussion_r1286278058


##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManagerUtil.java:
##########
@@ -411,4 +438,292 @@ public static Set<String> getSideInputStoreNames(
     }
     return sideInputStores;
   }
+
+  public static List<Future<Void>> initAndRestoreTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, SamzaContainerMetrics samzaContainerMetrics,
+      CheckpointManager checkpointManager, JobContext jobContext, ContainerModel containerModel,
+      Map<TaskName, Checkpoint> taskCheckpoints, Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames,
+      Config config, ExecutorService executor, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      File loggerStoreDir, Map<String, SystemConsumer> storeConsumers) {
+
+    Set<String> forceRestoreTasks = new HashSet<>();
+    // Initialize each TaskStorageManager.
+    taskRestoreManagers.forEach((taskName, restoreManagers) ->
+        restoreManagers.forEach((factoryName, taskRestoreManager) -> {
+          try {
+            taskRestoreManager.init(taskCheckpoints.get(taskName));
+          } catch (SamzaException ex) {
+            if (isUnwrappedExceptionDeletedException(ex) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+              // Get deleted SnapshotIndex blob with GetDeleted and mark the task to be restored with GetDeleted as well.
+              // this ensures that the restore downloads the snapshot, recreates a new snapshot, uploads it to blob store
+              // and creates a new checkpoint.
+              ((BlobStoreRestoreManager) taskRestoreManager).init(taskCheckpoints.get(taskName), true);
+              forceRestoreTasks.add(taskName.getTaskName());
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("init failed for task: %s with GetDeleted set to true", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex);
+            }
+          }
+        })
+    );
+
+    // Start each store consumer once.
+    // Note: These consumers are per system and only changelog system store consumers will be started.
+    // Some TaskRestoreManagers may not require the consumer to be started, but due to the agnostic nature of
+    // ContainerStorageManager we always start the changelog consumer here in case it is required
+    storeConsumers.values().stream().distinct().forEach(SystemConsumer::start);
+
+    return restoreAllTaskInstances(taskRestoreManagers, taskCheckpoints, taskBackendFactoryToStoreNames, jobContext,
+        containerModel, samzaContainerMetrics, checkpointManager, config, taskInstanceMetrics, executor, loggerStoreDir,
+        forceRestoreTasks);
+  }
+
+  /**
+   * Restores all TaskInstances and returns a future for each TaskInstance restore. If this restore fails with DeletedException
+   * it will retry the restore with getDeleted flag, get all the blobs, recreate a new checkpoint in blob store and
+   * write that checkpoint to checkpoint topic.
+   * @param taskRestoreManagers map of TaskName to factory name to TaskRestoreManager map.
+   */
+  public static List<Future<Void>> restoreAllTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, Map<TaskName, Checkpoint> taskCheckpoints,
+      Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames, JobContext jobContext,
+      ContainerModel containerModel, SamzaContainerMetrics samzaContainerMetrics, CheckpointManager checkpointManager,
+      Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics, ExecutorService executor,
+      File loggedStoreDir, Set<String> forceRestoreTask) {
+
+    List<Future<Void>> taskRestoreFutures = new ArrayList<>();
+
+    // Submit restore callable for each taskInstance
+    taskRestoreManagers.forEach((taskInstanceName, restoreManagersMap) -> {
+      // Submit for each restore factory
+      restoreManagersMap.forEach((factoryName, taskRestoreManager) -> {
+        long startTime = System.currentTimeMillis();
+        String taskName = taskInstanceName.getTaskName();
+        LOG.info("Starting restore for state for task: {}", taskName);
+
+        CompletableFuture<Void> restoreFuture;
+        if (forceRestoreTask.contains(taskName) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+          restoreFuture = ((BlobStoreRestoreManager) taskRestoreManager).restore(true);
+        } else {
+          restoreFuture = taskRestoreManager.restore();
+        }
+
+        CompletableFuture<Void> taskRestoreFuture = restoreFuture.handle((res, ex) -> {
+          updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+
+          if (ex != null) {
+            if (isUnwrappedExceptionDeletedException(ex)) {
+              LOG.warn(
+                  "Restore state for task: {} received DeletedException. Reattempting with getDeletedBlobs enabled",
+                  taskInstanceName.getTaskName());
+
+              // Try to restore with getDeleted flag
+              CompletableFuture<Void> future =
+                  restoreTaskStoresAndCreateCheckpointWithGetDeleted(taskInstanceName, taskCheckpoints, checkpointManager,
+                      taskRestoreManager, config, taskInstanceMetrics, executor,
+                      taskBackendFactoryToStoreNames.get(taskInstanceName).get(factoryName), loggedStoreDir, jobContext,
+                      containerModel);
+              try {
+                if (future != null) {
+                  future.join(); // Block and restore deleted state before continuing
+                }
+              } catch (Exception e) {
+                String msg = String.format("Unable to recover from DeletedException for task %s.", taskName);
+                throw new SamzaException(msg, e);
+              } finally {
+                updateRestoreTime(startTime, samzaContainerMetrics, taskInstanceName);
+              }
+            } else {
+              // log and rethrow exception to communicate restore failure
+              String msg = String.format("Error restoring state for task: %s", taskName);
+              LOG.error(msg, ex);
+              throw new SamzaException(msg, ex); // wrap in unchecked exception to throw from lambda
+            }
+          }
+
+          // Stop all persistent stores after restoring. Certain persistent stores opened in BulkLoad mode are compacted
+          // on stop, so paralleling stop() also parallelizes their compaction (a time-intensive operation).
+          try {
+            taskRestoreManager.close();
+          } catch (Exception e) {
+            LOG.error("Error closing restore manager for task: {} after {} restore", taskName,
+                ex != null ? "unsuccessful" : "successful", e);
+            // ignore exception from close. container may still be able to continue processing/backups
+            // if restore manager close fails.
+          }
+          return null;
+        });
+        taskRestoreFutures.add(taskRestoreFuture);
+      });
+    });
+
+    return taskRestoreFutures;
+  }
+
+  public static CompletableFuture<Void> restoreTaskStoresAndCreateCheckpointWithGetDeleted(TaskName taskName,
+      Map<TaskName, Checkpoint> taskCheckpoints, CheckpointManager checkpointManager,
+      TaskRestoreManager taskRestoreManager, Config config, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      ExecutorService executor, Set<String> storesToRestore, File loggedStoreBaseDirectory, JobContext jobContext,
+      ContainerModel containerModel) {
+
+    // if taskInstanceMetrics are specified use those for store metrics,
+    // otherwise (in case of StorageRecovery) use a blank MetricsRegistryMap
+    MetricsRegistry metricsRegistry =
+        taskInstanceMetrics.get(taskName) != null ? taskInstanceMetrics.get(taskName).registry()
+            : new MetricsRegistryMap();
+
+    BlobStoreManager blobStoreManager = getBlobStoreManager(config, executor);
+    JobConfig jobConfig = new JobConfig(config);
+    BlobStoreUtil blobStoreUtil =
+        new BlobStoreUtil(blobStoreManager, executor, new BlobStoreConfig(config), null,
+            new BlobStoreRestoreManagerMetrics(metricsRegistry));
+
+    BlobStoreRestoreManager blobStoreRestoreManager = (BlobStoreRestoreManager) taskRestoreManager;
+
+    CheckpointId checkpointId = CheckpointId.create();
+    Map<String, String> oldSCMs = ((CheckpointV2) taskCheckpoints.get(taskName))
+        .getStateCheckpointMarkers().get(BlobStoreStateBackendFactory.class.getName());
+
+    // Returns a single future that guarantees all the following are completed, in this order:
+    // 1. Restore state locally by getting deleted blobs from the blob store.
+    // 2. Create a new Checkpoint from restored state and back it up on the blob store.

Review Comment:
   Updated



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [samza] shekhars-li commented on a diff in pull request #1676: [SAMZA-2787] GetDeleted API and Recover from DeletedException

Posted by "shekhars-li (via GitHub)" <gi...@apache.org>.
shekhars-li commented on code in PR #1676:
URL: https://github.com/apache/samza/pull/1676#discussion_r1275550987


##########
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManagerUtil.java:
##########
@@ -411,4 +438,292 @@ public static Set<String> getSideInputStoreNames(
     }
     return sideInputStores;
   }
+
+  public static List<Future<Void>> initAndRestoreTaskInstances(
+      Map<TaskName, Map<String, TaskRestoreManager>> taskRestoreManagers, SamzaContainerMetrics samzaContainerMetrics,
+      CheckpointManager checkpointManager, JobContext jobContext, ContainerModel containerModel,
+      Map<TaskName, Checkpoint> taskCheckpoints, Map<TaskName, Map<String, Set<String>>> taskBackendFactoryToStoreNames,
+      Config config, ExecutorService executor, Map<TaskName, TaskInstanceMetrics> taskInstanceMetrics,
+      File loggerStoreDir, Map<String, SystemConsumer> storeConsumers) {
+
+    Set<String> forceRestoreTasks = new HashSet<>();
+    // Initialize each TaskStorageManager.
+    taskRestoreManagers.forEach((taskName, restoreManagers) ->
+        restoreManagers.forEach((factoryName, taskRestoreManager) -> {
+          try {
+            taskRestoreManager.init(taskCheckpoints.get(taskName));
+          } catch (SamzaException ex) {
+            if (isUnwrappedExceptionDeletedException(ex) && taskRestoreManager instanceof BlobStoreRestoreManager) {
+              // Get deleted SnapshotIndex blob with GetDeleted and mark the task to be restored with GetDeleted as well.
+              // this ensures that the restore downloads the snapshot, recreates a new snapshot, uploads it to blob store
+              // and creates a new checkpoint.
+              ((BlobStoreRestoreManager) taskRestoreManager).init(taskCheckpoints.get(taskName), true);

Review Comment:
   Confirmed that calling it twice is safe in our Impl of BlobStoreManager.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@samza.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org