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