prateekm commented on a change in pull request #1501:
URL: https://github.com/apache/samza/pull/1501#discussion_r636546734



##########
File path: 
samza-core/src/main/java/org/apache/samza/storage/blobstore/metrics/BlobStoreRestoreManagerMetrics.java
##########
@@ -0,0 +1,75 @@
+package org.apache.samza.storage.blobstore.metrics;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.samza.metrics.Counter;
+import org.apache.samza.metrics.Gauge;
+import org.apache.samza.metrics.MetricsRegistry;
+import org.apache.samza.metrics.Timer;
+
+
+public class BlobStoreRestoreManagerMetrics {
+  private static final String GROUP = 
BlobStoreRestoreManagerMetrics.class.getName();
+  private final MetricsRegistry metricsRegistry;
+
+  // ToDo per-task throughput

Review comment:
       Update TODO format (here and everywhere else).

##########
File path: 
samza-core/src/main/java/org/apache/samza/storage/blobstore/BlobStoreBackupManager.java
##########
@@ -0,0 +1,349 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.storage.blobstore;
+
+import com.google.common.collect.ImmutableMap;
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionStage;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.samza.SamzaException;
+import org.apache.samza.checkpoint.Checkpoint;
+import org.apache.samza.checkpoint.CheckpointId;
+import org.apache.samza.config.BlobStoreConfig;
+import org.apache.samza.config.Config;
+import org.apache.samza.config.JobConfig;
+import org.apache.samza.config.StorageConfig;
+import org.apache.samza.job.model.ContainerModel;
+import org.apache.samza.job.model.JobModel;
+import org.apache.samza.job.model.TaskModel;
+import org.apache.samza.storage.StorageManagerUtil;
+import org.apache.samza.storage.TaskBackupManager;
+import org.apache.samza.storage.blobstore.diff.DirDiff;
+import org.apache.samza.storage.blobstore.index.DirIndex;
+import org.apache.samza.storage.blobstore.index.SnapshotIndex;
+import org.apache.samza.storage.blobstore.index.SnapshotMetadata;
+import 
org.apache.samza.storage.blobstore.metrics.BlobStoreBackupManagerMetrics;
+import org.apache.samza.storage.blobstore.util.BlobStoreUtil;
+import org.apache.samza.storage.blobstore.util.DirDiffUtil;
+import org.apache.samza.util.Clock;
+import org.apache.samza.util.FutureUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class BlobStoreBackupManager implements TaskBackupManager {
+  private static final Logger LOG = 
LoggerFactory.getLogger(BlobStoreBackupManager.class);
+
+  private final JobModel jobModel;
+  private final ExecutorService executor;
+  private final String jobName;
+  private final String jobId;
+  private final ContainerModel containerModel;
+  private final TaskModel taskModel;
+  private final String taskName;
+  private final Config config;
+  private final Clock clock;
+  private final StorageManagerUtil storageManagerUtil;
+  private final List<String> storesToBackup;
+  private final File loggedStoreBaseDir;
+  private final BlobStoreUtil blobStoreUtil;
+
+  private final BlobStoreBackupManagerMetrics metrics;
+
+  /**
+   * Map of store name to a Pair of blob id of {@link SnapshotIndex} and the 
corresponding {@link SnapshotIndex} from
+   * last successful task checkpoint or {@link #upload}.
+   *
+   * After {@link #init}, the map reflects the contents of the last completed 
checkpoint for the task from the previous
+   * deployment, if any.
+   *
+   * During regular processing, this map is updated after each successful 
{@link #upload} with the blob id of
+   * {@link SnapshotIndex} and the corresponding {@link SnapshotIndex} of the 
upload.
+   *
+   * The contents of this map are used to calculate the diff for local state 
between the last and the current checkpoint
+   * during {@link #upload}.
+   *
+   * Since the task commit process guarantees that the async stage of the 
previous commit is complete before another
+   * commit can start, this future is guaranteed to be complete in the call to 
{@link #upload} during the next commit.
+   *
+   * This field is non-final, since the future itself is replaced in its 
entirety after init/upload.
+   * The internal map contents are never directly modified (e.g. using puts). 
It's volatile to ensure visibility
+   * across threads since the map assignment may happen on a different thread 
than the one reading the contents.
+   */
+  private volatile CompletableFuture<Map<String, Pair<String, SnapshotIndex>>>
+      prevStoreSnapshotIndexesFuture;
+
+  public BlobStoreBackupManager(JobModel jobModel, ContainerModel 
containerModel, TaskModel taskModel,
+      ExecutorService backupExecutor, BlobStoreBackupManagerMetrics 
blobStoreTaskBackupMetrics, Config config,
+      Clock clock, File loggedStoreBaseDir, StorageManagerUtil 
storageManagerUtil, BlobStoreUtil blobStoreUtil) {
+    this.jobModel = jobModel;
+    this.jobName = new JobConfig(config).getName().get();
+    this.jobId = new JobConfig(config).getJobId();
+    this.containerModel = containerModel;
+    this.taskModel = taskModel;
+    this.taskName = taskModel.getTaskName().getTaskName();
+    this.executor = backupExecutor;
+    this.config = config;
+    this.clock = clock;
+    this.storageManagerUtil = storageManagerUtil;
+    BlobStoreConfig blobStoreConfig = new BlobStoreConfig(config);
+    StorageConfig storageConfig = new StorageConfig(config);
+    this.storesToBackup =
+        
blobStoreConfig.getStoresWithStateBackendBackupFactory(storageConfig.getStoreNames(),
+            BlobStoreStateBackendFactory.class.getName());
+    this.loggedStoreBaseDir = loggedStoreBaseDir;
+    this.blobStoreUtil = blobStoreUtil;
+    this.prevStoreSnapshotIndexesFuture = 
CompletableFuture.completedFuture(ImmutableMap.of());
+    this.metrics = blobStoreTaskBackupMetrics;
+    metrics.initStoreMetrics(storesToBackup);
+  }
+
+  @Override
+  public void init(Checkpoint checkpoint) {
+    long startTime = System.nanoTime();
+    LOG.debug("Initializing blob store backup manager for task: {}", taskName);
+
+    blobStoreUtil.initBlobStoreManager();
+
+    // Note: blocks the caller thread.
+    Map<String, Pair<String, SnapshotIndex>> prevStoreSnapshotIndexes =
+        blobStoreUtil.getStoreSnapshotIndexes(jobName, jobId, taskName, 
checkpoint);
+    this.prevStoreSnapshotIndexesFuture =
+        
CompletableFuture.completedFuture(ImmutableMap.copyOf(prevStoreSnapshotIndexes));
+    metrics.initNs.set(System.nanoTime() - startTime);
+  }
+
+  @Override
+  public Map<String, String> snapshot(CheckpointId checkpointId) {
+    // No-op. Stores are flushed and checkpoints are created by commit manager
+    return Collections.emptyMap();
+  }
+
+  @Override
+  public CompletableFuture<Map<String, String>> upload(CheckpointId 
checkpointId, Map<String, String> storeSCMs) {
+    long uploadStartTime = System.nanoTime();
+
+    // reset gauges for each upload
+    metrics.filesToUpload.getValue().set(0L);
+    metrics.bytesToUpload.getValue().set(0L);
+    metrics.filesUploaded.getValue().set(0L);
+    metrics.bytesUploaded.getValue().set(0L);
+    metrics.filesRemaining.getValue().set(0L);
+    metrics.bytesRemaining.getValue().set(0L);
+    metrics.filesToRetain.getValue().set(0L);
+    metrics.bytesToRetain.getValue().set(0L);
+
+    // This map is used to atomically replace the 
prevStoreSnapshotIndexesFuture map at the end of the task commit
+    Map<String, CompletableFuture<Pair<String, SnapshotIndex>>>
+        storeToSCMAndSnapshotIndexPairFutures = new HashMap<>();
+    // This map is used to return serialized State Checkpoint Markers to the 
caller
+    Map<String, CompletableFuture<String>> storeToSerializedSCMFuture = new 
HashMap<>();
+
+    storesToBackup.forEach((storeName) -> {
+      long storeUploadStartTime = System.nanoTime();
+      try {
+        // metadata for the current store snapshot to upload
+        SnapshotMetadata snapshotMetadata = new SnapshotMetadata(checkpointId, 
jobName, jobId, taskName, storeName);
+
+        // get the local store dir corresponding to the current checkpointId
+        File storeDir = storageManagerUtil.getTaskStoreDir(loggedStoreBaseDir, 
storeName,
+            taskModel.getTaskName(), taskModel.getTaskMode());
+        String checkpointDirPath = 
storageManagerUtil.getStoreCheckpointDir(storeDir, checkpointId);
+        File checkpointDir = new File(checkpointDirPath);
+
+        LOG.debug("Got task: {} store: {} storeDir: {} and checkpointDir: {}",
+            taskName, storeName, storeDir, checkpointDir);
+
+        // guaranteed to be available since a new task commit may not start 
until the previous one is complete
+        Map<String, Pair<String, SnapshotIndex>> prevStoreSnapshotIndexes =
+            prevStoreSnapshotIndexesFuture.get(0, TimeUnit.MILLISECONDS);
+
+        // get the previous store directory contents
+        DirIndex prevDirIndex;
+
+        if (prevStoreSnapshotIndexes.containsKey(storeName)) {
+          prevDirIndex = 
prevStoreSnapshotIndexes.get(storeName).getRight().getDirIndex();
+        } else {
+          // no previous SnapshotIndex means that this is the first commit for 
this store. Create an empty DirIndex.
+          prevDirIndex = new DirIndex(checkpointDir.getName(), 
Collections.emptyList(), Collections.emptyList(),
+              Collections.emptyList(), Collections.emptyList());
+        }
+
+        long dirDiffStartTime = System.nanoTime();
+        // get the diff between previous and current store directories
+        DirDiff dirDiff = DirDiffUtil.getDirDiff(checkpointDir, prevDirIndex, 
DirDiffUtil.areSameFile(false));
+        metrics.storeDirDiffNs.get(storeName).update(System.nanoTime() - 
dirDiffStartTime);
+
+        DirDiff.Stats stats = DirDiff.getStats(dirDiff);
+        updateStoreDiffMetrics(storeName, stats);
+        metrics.filesToUpload.getValue().addAndGet(stats.filesAdded);
+        metrics.bytesToUpload.getValue().addAndGet(stats.bytesAdded);
+        // Note: FilesRemaining metric is set to FilesAdded in the beginning 
of the current upload and then counted down
+        // for each upload.
+        metrics.filesRemaining.getValue().addAndGet(stats.filesAdded);
+        metrics.bytesRemaining.getValue().addAndGet(stats.bytesAdded);
+        metrics.filesToRetain.getValue().addAndGet(stats.filesRetained);
+        metrics.bytesToRetain.getValue().addAndGet(stats.bytesRetained);
+
+        // upload the diff to the blob store and get the new directory index
+        CompletionStage<DirIndex> dirIndexFuture = 
blobStoreUtil.putDir(dirDiff, snapshotMetadata);
+
+        CompletionStage<SnapshotIndex> snapshotIndexFuture =
+            dirIndexFuture.thenApplyAsync(dirIndex -> {
+              LOG.trace("Dir upload complete. Returning new SnapshotIndex for 
task: {} store: {}.", taskName, storeName);
+              Optional<String> prevSnapshotIndexBlobId =
+                  
Optional.ofNullable(prevStoreSnapshotIndexes.get(storeName)).map(Pair::getLeft);
+              return new SnapshotIndex(clock.currentTimeMillis(), 
snapshotMetadata, dirIndex, prevSnapshotIndexBlobId);
+            }, executor);
+
+        // upload the new snapshot index to the blob store and get its blob id
+        CompletionStage<String> snapshotIndexBlobIdFuture =
+            snapshotIndexFuture
+                .thenComposeAsync(si -> {
+                  LOG.trace("Uploading Snapshot index for task: {} store: {}", 
taskName, storeName);
+                  return blobStoreUtil.putSnapshotIndex(si);
+                }, executor);
+
+        // update the map of storeName to previous snapshot index 
storeToSCMAndSnapshotIndexPairFutures which is temporary

Review comment:
       Run on sentence is hard to parse, can you simplify? Don't need to 
mention variable names in comment, it's obvious from context. Just say what is 
happening here and why. 
   
   There is actually no update happening here at all, is this comment misplaced 
(should be on 244)?
   
   Maybe we can delete the comment entirely?

##########
File path: 
samza-core/src/main/java/org/apache/samza/storage/blobstore/util/DirDiffUtil.java
##########
@@ -0,0 +1,404 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.samza.storage.blobstore.util;
+
+import com.google.common.base.Preconditions;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.nio.file.attribute.PosixFileAttributes;
+import java.nio.file.attribute.PosixFilePermissions;
+import java.util.HashMap;
+import java.util.zip.CRC32;
+import java.util.zip.CheckedInputStream;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.samza.SamzaException;
+import org.apache.samza.storage.blobstore.diff.DirDiff;
+import org.apache.samza.storage.blobstore.index.DirIndex;
+import org.apache.samza.storage.blobstore.index.FileIndex;
+
+import java.io.File;

Review comment:
       Sort imports (you can do this for entire directory at once in IntelliJ 
(Right click - > Optimize imports).




-- 
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.

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to