Support blob manifest backup for fdbbackup cmdline (#10091)

This commit is contained in:
Hui Liu 2023-05-08 16:07:22 -07:00 committed by GitHub
parent f1e22f9a1e
commit 53e68065e7
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
14 changed files with 177 additions and 58 deletions

View File

@ -193,6 +193,9 @@ enum {
OPT_DSTONLY,
OPT_TRACE_FORMAT,
// blob granules backup/restore
OPT_BLOB_MANIFEST_URL,
};
// Top level binary commands.
@ -280,6 +283,7 @@ CSimpleOpt::SOption g_rgBackupStartOptions[] = {
{ OPT_INCREMENTALONLY, "--incremental", SO_NONE },
{ OPT_ENCRYPTION_KEY_FILE, "--encryption-key-file", SO_REQ_SEP },
{ OPT_ENCRYPT_FILES, "--encrypt-files", SO_REQ_SEP },
{ OPT_BLOB_MANIFEST_URL, "--blob-manifest-url", SO_REQ_SEP },
TLS_OPTION_FLAGS,
SO_END_OF_OPTIONS
};
@ -1131,6 +1135,10 @@ static void printBackupUsage(bool devhelp) {
"either enable (1) or disable (0) encryption at rest with snapshot backups. This option refers to block "
"level encryption of snapshot backups while --encryption-key-file (above) refers to file level encryption. "
"Generally, these two options should not be used together.\n");
printf(" --blob-manifest-url URL\n"
" Perform blob manifest backup. Manifest files are stored to the destination URL.\n"
" Blob granules should be enabled first for manifest backup.\n");
printf(TLS_HELP);
printf(" -w, --wait Wait for the backup to complete (allowed with `start' and `discontinue').\n");
printf(" -z, --no-stop-when-done\n"
@ -1965,7 +1973,8 @@ ACTOR Future<Void> submitBackup(Database db,
WaitForComplete waitForCompletion,
StopWhenDone stopWhenDone,
UsePartitionedLog usePartitionedLog,
IncrementalBackupOnly incrementalBackupOnly) {
IncrementalBackupOnly incrementalBackupOnly,
Optional<std::string> blobManifestUrl) {
try {
state FileBackupAgent backupAgent;
ASSERT(!backupRanges.empty());
@ -2018,7 +2027,9 @@ ACTOR Future<Void> submitBackup(Database db,
encryptionEnabled,
stopWhenDone,
usePartitionedLog,
incrementalBackupOnly));
incrementalBackupOnly,
{},
blobManifestUrl));
// Wait for the backup to complete, if requested
if (waitForCompletion) {
@ -3531,6 +3542,7 @@ int main(int argc, char* argv[]) {
bool jsonOutput = false;
DeleteData deleteData{ false };
Optional<std::string> encryptionKeyFile;
Optional<std::string> blobManifestUrl;
BackupModifyOptions modifyOptions;
@ -3953,6 +3965,10 @@ int main(int argc, char* argv[]) {
case OPT_JSON:
jsonOutput = true;
break;
case OPT_BLOB_MANIFEST_URL: {
blobManifestUrl = args->OptionArg();
break;
}
}
}
@ -4185,7 +4201,8 @@ int main(int argc, char* argv[]) {
waitForDone,
stopWhenDone,
usePartitionedLog,
incrementalBackupOnly));
incrementalBackupOnly,
blobManifestUrl));
break;
}
@ -4371,6 +4388,7 @@ int main(int argc, char* argv[]) {
onlyApplyMutationLogs,
inconsistentSnapshotOnly,
encryptionKeyFile));
break;
case RestoreType::WAIT:
f = stopAfter(success(ba.waitRestore(db, KeyRef(tagName), Verbose::True)));

View File

@ -18,10 +18,12 @@
* limitations under the License.
*/
#include "fdbclient/BlobGranuleCommon.h"
#include "fdbclient/CommitProxyInterface.h"
#include "fdbclient/DatabaseConfiguration.h"
#include "fdbclient/TenantEntryCache.actor.h"
#include "fdbclient/TenantManagement.actor.h"
#include "fdbclient/BlobRestoreCommon.h"
#include "fdbrpc/TenantInfo.h"
#include "fdbrpc/simulator.h"
#include "flow/EncryptUtils.h"
@ -5291,7 +5293,8 @@ public:
StopWhenDone stopWhenDone,
UsePartitionedLog partitionedLog,
IncrementalBackupOnly incrementalBackupOnly,
Optional<std::string> encryptionKeyFileName) {
Optional<std::string> encryptionKeyFileName,
Optional<std::string> blobManifestUrl) {
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
tr->setOption(FDBTransactionOptions::COMMIT_ON_FIRST_PROXY);
@ -5349,6 +5352,21 @@ public:
throw backup_error();
}
if (blobManifestUrl.present()) {
state BlobGranuleBackupConfig bgBackupConfig;
bool enabled = wait(bgBackupConfig.enabled().getD(tr));
if (enabled) {
fprintf(stderr, "ERROR: Abort existing blob manifest backup first before creating new one.\n");
throw backup_error();
} else {
bgBackupConfig.manifestUrl().set(tr, blobManifestUrl.get());
bgBackupConfig.mutationLogsUrl().set(tr, bc->getURL());
bgBackupConfig.enabled().set(tr, true);
}
// Allow only incremental backup
incrementalBackupOnly = IncrementalBackupOnly::True;
}
KeyRangeMap<int> backupRangeSet;
for (auto& backupRange : backupRanges) {
backupRangeSet.insert(backupRange, 1);
@ -5411,6 +5429,7 @@ public:
config.partitionedLogEnabled().set(tr, partitionedLog);
config.incrementalBackupOnly().set(tr, incrementalBackupOnly);
config.enableSnapshotBackupEncryption().set(tr, encryptionEnabled);
config.blobBackupEnabled().set(tr, blobManifestUrl.present());
Key taskKey = wait(fileBackup::StartFullBackupTaskFunc::addTask(
tr, backupAgent->taskBucket, uid, TaskCompletionKey::noSignal()));
@ -5516,6 +5535,8 @@ public:
// this also sets restore.add/removePrefix.
restore.initApplyMutations(tr, addPrefix, removePrefix, onlyApplyMutationLogs);
//
Key taskKey = wait(fileBackup::StartFullRestoreTaskFunc::addTask(
tr, backupAgent->taskBucket, uid, TaskCompletionKey::noSignal()));
@ -5649,6 +5670,12 @@ public:
.detail("TagName", tagName.c_str())
.detail("Status", BackupAgentBase::getStateText(status));
bool bgBackupEnabled = wait(config.blobBackupEnabled().getD(tr));
if (bgBackupEnabled) {
BlobGranuleBackupConfig bgbackupConfig;
bgbackupConfig.enabled().set(tr, false);
}
// Cancel backup task through tag
wait(tag.cancel(tr));
@ -6171,6 +6198,7 @@ public:
inconsistentSnapshotOnly,
beginVersion,
randomUid));
wait(tr->commit());
break;
} catch (Error& e) {
@ -6687,7 +6715,8 @@ Future<Void> FileBackupAgent::submitBackup(Reference<ReadYourWritesTransaction>
StopWhenDone stopWhenDone,
UsePartitionedLog partitionedLog,
IncrementalBackupOnly incrementalBackupOnly,
Optional<std::string> const& encryptionKeyFileName) {
Optional<std::string> const& encryptionKeyFileName,
Optional<std::string> const& blobManifestUrl) {
return FileBackupAgentImpl::submitBackup(this,
tr,
outContainer,
@ -6700,7 +6729,8 @@ Future<Void> FileBackupAgent::submitBackup(Reference<ReadYourWritesTransaction>
stopWhenDone,
partitionedLog,
incrementalBackupOnly,
encryptionKeyFileName);
encryptionKeyFileName,
blobManifestUrl);
}
Future<Void> FileBackupAgent::discontinueBackup(Reference<ReadYourWritesTransaction> tr, Key tagName) {

View File

@ -276,7 +276,8 @@ public:
StopWhenDone = StopWhenDone::True,
UsePartitionedLog = UsePartitionedLog::False,
IncrementalBackupOnly = IncrementalBackupOnly::False,
Optional<std::string> const& encryptionKeyFileName = {});
Optional<std::string> const& encryptionKeyFileName = {},
Optional<std::string> const& blobManifestUrl = {});
Future<Void> submitBackup(Database cx,
Key outContainer,
Optional<std::string> proxy,
@ -288,7 +289,8 @@ public:
StopWhenDone stopWhenDone = StopWhenDone::True,
UsePartitionedLog partitionedLog = UsePartitionedLog::False,
IncrementalBackupOnly incrementalBackupOnly = IncrementalBackupOnly::False,
Optional<std::string> const& encryptionKeyFileName = {}) {
Optional<std::string> const& encryptionKeyFileName = {},
Optional<std::string> const& blobManifestUrl = {}) {
return runRYWTransactionFailIfLocked(cx, [=](Reference<ReadYourWritesTransaction> tr) {
return submitBackup(tr,
outContainer,
@ -301,7 +303,8 @@ public:
stopWhenDone,
partitionedLog,
incrementalBackupOnly,
encryptionKeyFileName);
encryptionKeyFileName,
blobManifestUrl);
});
}
@ -960,6 +963,8 @@ public:
return updateErrorInfo(cx, e, details);
}
KeyBackedProperty<bool> blobBackupEnabled() { return configSpace.pack(__FUNCTION__sr); }
};
// Helper class for reading restore data from a buffer and throwing the right errors.

View File

@ -25,7 +25,7 @@
#include "fdbclient/BlobCipher.h"
#include "fdbclient/CommitTransaction.h"
#include "fdbclient/FDBTypes.h"
#include "fdbclient/KeyBackedTypes.actor.h"
#include "flow/EncryptUtils.h"
#include "flow/IRandom.h"
#include "flow/serialize.h"

View File

@ -0,0 +1,38 @@
/*
* BlobRestoreCommon.h
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2022 Apple Inc. and the FoundationDB project authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#ifndef FDBCLIENT_BLOBRESTORECOMMON_H
#define FDBCLIENT_BLOBRESTORECOMMON_H
#pragma once
#include "fdbclient/FDBTypes.h"
#include "fdbclient/SystemData.h"
#include "fdbclient/KeyBackedTypes.actor.h"
struct BlobGranuleBackupConfig : public KeyBackedClass {
BlobGranuleBackupConfig(KeyRef prefix = SystemKey("\xff\x02/bgbackup/"_sr)) : KeyBackedClass(prefix) {}
KeyBackedProperty<bool> enabled() { return subspace.pack(__FUNCTION__sr); }
KeyBackedProperty<std::string> manifestUrl() { return subspace.pack(__FUNCTION__sr); }
KeyBackedProperty<std::string> mutationLogsUrl() { return subspace.pack(__FUNCTION__sr); }
};
#endif

View File

@ -46,6 +46,7 @@
#include "fdbclient/SystemData.h"
#include "fdbclient/Tuple.h"
#include "fdbclient/BackupAgent.actor.h"
#include "fdbclient/BlobRestoreCommon.h"
#include "fdbserver/BlobManagerInterface.h"
#include "fdbserver/Knobs.h"
#include "fdbserver/BlobGranuleValidation.actor.h"
@ -452,7 +453,6 @@ struct BlobManagerData : NonCopyable, ReferenceCounted<BlobManagerData> {
fmt::print("BM {} constructed backup container\n", epoch);
}
}
if (SERVER_KNOBS->BLOB_RESTORE_MANIFEST_URL != "") {
manifestStore = BlobConnectionProvider::newBlobConnectionProvider(SERVER_KNOBS->BLOB_RESTORE_MANIFEST_URL);
}
@ -3565,10 +3565,6 @@ ACTOR Future<Void> recoverBlobManager(Reference<BlobManagerData> bmData) {
bool isFullRestore = wait(BlobRestoreController::isRestoring(restoreController));
bmData->isFullRestoreMode = isFullRestore;
if (bmData->isFullRestoreMode) {
if (!bmData->manifestStore.isValid()) {
TraceEvent(SevError, "InvalidBlobManifestUrl").detail("Url", SERVER_KNOBS->BLOB_RESTORE_MANIFEST_URL);
throw blob_restore_invalid_manifest_url();
}
Optional<BlobRestoreState> restoreState = wait(BlobRestoreController::getState(restoreController));
ASSERT(restoreState.present());
state BlobRestorePhase phase = restoreState.get().phase;
@ -5510,8 +5506,9 @@ ACTOR Future<int64_t> getLastFlushTs(Database db) {
}
}
bool shouldBackupManifest(Reference<BlobManagerData> bmData) {
if (!SERVER_KNOBS->BLOB_MANIFEST_BACKUP) {
ACTOR Future<bool> shouldBackupManifest(Reference<BlobManagerData> bmData) {
bool enabled = wait(BlobGranuleBackupConfig().enabled().getD(SystemDBWriteLockedNow(bmData->db.getReference())));
if (!enabled) {
return false;
}
@ -5531,12 +5528,12 @@ bool shouldBackupManifest(Reference<BlobManagerData> bmData) {
ACTOR Future<Void> truncateMutationLogs(Reference<BlobManagerData> bmData) {
loop {
wait(delay(SERVER_KNOBS->BLOB_MANIFEST_BACKUP_INTERVAL));
if (!shouldBackupManifest(bmData)) {
continue;
}
try {
bool shouldBackup = wait(shouldBackupManifest(bmData));
if (!shouldBackup) {
continue;
}
int64_t lastFlushTs = wait(getLastFlushTs(bmData->db));
bool shouldFlush = (now() - lastFlushTs) > SERVER_KNOBS->BLOB_RESTORE_MLOGS_RETENTION_SECS;
if (!shouldFlush) {
@ -5589,6 +5586,12 @@ ACTOR Future<Void> truncateMutationLogs(Reference<BlobManagerData> bmData) {
}
}
ACTOR Future<Reference<BlobConnectionProvider>> initManifestStore(Reference<BlobManagerData> bmData) {
auto db = SystemDBWriteLockedNow(bmData->db.getReference());
std::string url = wait(BlobGranuleBackupConfig().manifestUrl().getD(db));
return BlobConnectionProvider::newBlobConnectionProvider(url);
}
ACTOR Future<Void> backupManifest(Reference<BlobManagerData> bmData) {
bmData->initBStore();
@ -5596,22 +5599,25 @@ ACTOR Future<Void> backupManifest(Reference<BlobManagerData> bmData) {
bmData->enableManifestEncryption = config.encryptionAtRestMode.isEncryptionEnabled();
loop {
// Skip backup if no active blob ranges
if (shouldBackupManifest(bmData)) {
if (bmData->manifestStore.isValid()) {
int64_t bytes = wait(dumpManifest(bmData->db,
bmData->dbInfo,
bmData->manifestStore,
bmData->epoch,
bmData->manifestDumperSeqNo,
bmData->enableManifestEncryption));
bmData->stats.lastManifestSeqNo = bmData->manifestDumperSeqNo;
bmData->stats.manifestSizeInBytes += bytes;
bmData->stats.lastManifestDumpTs = now();
bmData->manifestDumperSeqNo++;
} else {
TraceEvent(SevError, "InvalidBlobManifestUrl").detail("Url", SERVER_KNOBS->BLOB_RESTORE_MANIFEST_URL);
try {
// Skip backup if no active blob ranges
bool shouldBackup = wait(shouldBackupManifest(bmData));
if (shouldBackup) {
if (bmData->manifestStore.isValid()) {
int64_t bytes = wait(dumpManifest(bmData->db,
bmData->dbInfo,
bmData->manifestStore,
bmData->epoch,
bmData->manifestDumperSeqNo,
bmData->enableManifestEncryption));
bmData->stats.lastManifestSeqNo = bmData->manifestDumperSeqNo;
bmData->stats.manifestSizeInBytes += bytes;
bmData->stats.lastManifestDumpTs = now();
bmData->manifestDumperSeqNo++;
}
}
} catch (Error& e) {
TraceEvent("BackupManifestError").error(e);
}
wait(delay(SERVER_KNOBS->BLOB_MANIFEST_BACKUP_INTERVAL));
}
@ -5706,10 +5712,8 @@ ACTOR Future<Void> blobManager(BlobManagerInterface bmInterf,
if (SERVER_KNOBS->BG_ENABLE_MERGING) {
self->addActor.send(granuleMergeChecker(self));
}
if (SERVER_KNOBS->BLOB_MANIFEST_BACKUP) {
self->addActor.send(backupManifest(self));
self->addActor.send(truncateMutationLogs(self));
}
self->addActor.send(backupManifest(self));
self->addActor.send(truncateMutationLogs(self));
if (BUGGIFY && !self->isFullRestoreMode) {
self->addActor.send(chaosRangeMover(self));

View File

@ -144,7 +144,6 @@ public:
ASSERT(iter->seqNo == seqNo);
if (iter->segmentNo != nextSegmentNo) {
TraceEvent("BlobRestoreMissingSegment")
.detail("Url", SERVER_KNOBS->BLOB_RESTORE_MANIFEST_URL)
.detail("Epoch", epoch)
.detail("SeqNo", epoch)
.detail("Expected", nextSegmentNo)
@ -183,7 +182,7 @@ public:
}
dprint("No valid blob manifest files\n");
TraceEvent("BlobRestoreMissingManifest").detail("Url", SERVER_KNOBS->BLOB_RESTORE_MANIFEST_URL);
TraceEvent("BlobRestoreMissingManifest").log();
throw blob_restore_missing_manifest();
}
@ -644,7 +643,7 @@ private:
Reference<BackupContainerFileSystem> container,
BlobManifestFile tailerFile) {
if (tailerFile.segmentNo != 0) {
TraceEvent("BlobRestoreMissingTailer").detail("Url", SERVER_KNOBS->BLOB_RESTORE_MANIFEST_URL);
TraceEvent("BlobRestoreMissingTailer").log();
throw blob_restore_corrupted_manifest();
}

View File

@ -26,6 +26,7 @@
#include "fdbclient/BackupAgent.actor.h"
#include "fdbclient/BlobWorkerInterface.h"
#include "fdbclient/KeyBackedTypes.actor.h"
#include "fdbclient/BlobRestoreCommon.h"
#include "fdbserver/Status.actor.h"
#include "flow/ITrace.h"
#include "flow/ProtocolVersion.h"
@ -2431,6 +2432,7 @@ ACTOR static Future<JsonBuilderObject> clusterSummaryStatisticsFetcher(
}
ACTOR static Future<JsonBuilderObject> blobGranulesStatusFetcher(
Database cx,
Optional<BlobManagerInterface> managerIntf,
std::vector<BlobWorkerInterface> workers,
std::unordered_map<NetworkAddress, WorkerInterface> addressWorkersMap,
@ -2441,6 +2443,8 @@ ACTOR static Future<JsonBuilderObject> blobGranulesStatusFetcher(
statusObj["number_of_blob_workers"] = static_cast<int>(workers.size());
try {
bool backupEnabled = wait(BlobGranuleBackupConfig().enabled().getD(SystemDBWriteLockedNow(cx.getReference())));
statusObj["blob_granules_backup_enabled"] = backupEnabled;
// Blob manager status
if (managerIntf.present()) {
Optional<TraceEventFields> fields = wait(timeoutError(
@ -3500,10 +3504,10 @@ ACTOR Future<StatusReply> clusterGetStatus(
statusObj["clients"] = clientStatusFetcher(clientStatus);
if (configuration.present() && configuration.get().blobGranulesEnabled) {
JsonBuilderObject blobGranuelsStatus =
wait(timeoutError(blobGranulesStatusFetcher(
db->get().blobManager, blobWorkers, address_workers, &status_incomplete_reasons),
2.0));
JsonBuilderObject blobGranuelsStatus = wait(
timeoutError(blobGranulesStatusFetcher(
cx, db->get().blobManager, blobWorkers, address_workers, &status_incomplete_reasons),
2.0));
statusObj["blob_granules"] = blobGranuelsStatus;
JsonBuilderObject blobRestoreStatus =
wait(timeoutError(blobRestoreStatusFetcher(cx, &status_incomplete_reasons), 2.0));

View File

@ -28,6 +28,7 @@
#include "fdbclient/Knobs.h"
#include "fdbclient/SystemData.h"
#include "fdbclient/BlobGranuleReader.actor.h"
#include "fdbclient/BlobRestoreCommon.h"
#include "fdbserver/Knobs.h"
#include "fdbserver/workloads/workloads.actor.h"
#include "fdbserver/BlobGranuleServerCommon.actor.h"
@ -77,8 +78,7 @@ struct BlobRestoreWorkload : TestWorkload {
if (self->performRestore_) {
fmt::print("Perform blob restore\n");
// disable manifest backup and log truncation
KnobValueRef knobFalse = KnobValueRef::create(bool{ false });
IKnobCollection::getMutableGlobalKnobCollection().setKnob("blob_manifest_backup", knobFalse);
wait(disableManifestBackup(cx));
wait(store(self->restoreTargetVersion_, getRestoreVersion(cx, self)));
if (self->restoreTargetVersion_ == invalidVersion) {
@ -126,6 +126,16 @@ struct BlobRestoreWorkload : TestWorkload {
return targetVersion;
}
static Future<Void> disableManifestBackup(Database cx) {
return runRYWTransaction(cx, [](Reference<ReadYourWritesTransaction> tr) -> Future<Void> {
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE);
BlobGranuleBackupConfig().enabled().set(tr, false);
return Void();
});
}
// Start backup agent on the extra db
ACTOR Future<Void> runBackupAgent(BlobRestoreWorkload* self) {
state FileBackupAgent backupAgent;

View File

@ -46,6 +46,7 @@ struct IncrementalBackupWorkload : TestWorkload {
bool stopBackup;
bool checkBeginVersion;
bool clearBackupAgentKeys;
Standalone<StringRef> blobManifestUrl;
IncrementalBackupWorkload(WorkloadContext const& wcx) : TestWorkload(wcx) {
backupDir = getOption(options, "backupDir"_sr, "file://simfdb/backups/"_sr);
@ -57,6 +58,7 @@ struct IncrementalBackupWorkload : TestWorkload {
stopBackup = getOption(options, "stopBackup"_sr, false);
checkBeginVersion = getOption(options, "checkBeginVersion"_sr, false);
clearBackupAgentKeys = getOption(options, "clearBackupAgentKeys"_sr, false);
blobManifestUrl = getOption(options, "blobManifestUrl"_sr, ""_sr);
}
Future<Void> setup(Database const& cx) override { return Void(); }
@ -157,6 +159,10 @@ struct IncrementalBackupWorkload : TestWorkload {
if (self->submitOnly) {
TraceEvent("IBackupSubmitAttempt").log();
try {
Optional<std::string> blobManifestUrl;
if (!self->blobManifestUrl.empty()) {
blobManifestUrl = self->blobManifestUrl.toString();
}
wait(self->backupAgent.submitBackup(cx,
self->backupDir,
{},
@ -167,7 +173,9 @@ struct IncrementalBackupWorkload : TestWorkload {
true,
StopWhenDone::False,
UsePartitionedLog::False,
IncrementalBackupOnly::True));
IncrementalBackupOnly::True,
{},
blobManifestUrl));
} catch (Error& e) {
TraceEvent("IBackupSubmitError").error(e);
if (e.code() != error_code_backup_duplicate) {

View File

@ -8,7 +8,6 @@ storageEngineExcludeTypes = [4, 5]
[[knobs]]
bg_consistency_check_enabled = 0
blob_manifest_backup = true
shard_encode_location_metadata = false
bw_throttling_enabled = false
@ -22,7 +21,7 @@ waitForQuiescence = false
[[test.workload]]
testName = 'BlobRestoreWorkload'
setupBlob = true
[[test]]
testTitle = 'BackupMutationLogs'
simBackupAgents = 'BackupToFile'
@ -32,9 +31,10 @@ waitForQuiescence = false
[[test.workload]]
testName = 'IncrementalBackup'
tag = 'default'
blobManifestUrl = 'file://simfdb/fdbblob/manifest'
submitOnly = true
waitForBackup = true
[[test]]
testTitle = 'CycleTest'
simBackupAgents = 'BackupToFile'

View File

@ -22,7 +22,7 @@ waitForQuiescence = false
[[test.workload]]
testName = 'BlobRestoreWorkload'
setupBlob = true
[[test]]
testTitle = 'BackupMutationLogs'
simBackupAgents = 'BackupToFile'
@ -32,9 +32,10 @@ waitForQuiescence = false
[[test.workload]]
testName = 'IncrementalBackup'
tag = 'default'
blobManifestUrl = 'file://simfdb/fdbblob/manifest'
submitOnly = true
waitForBackup = true
[[test]]
testTitle = 'WriteTest'
simBackupAgents = 'BackupToFile'

View File

@ -27,6 +27,7 @@ waitForQuiescence = false
[[test.workload]]
testName = 'IncrementalBackup'
tag = 'default'
blobManifestUrl = 'file://simfdb/fdbblob/manifest'
submitOnly = true
waitForBackup = true

View File

@ -22,7 +22,7 @@ waitForQuiescence = false
[[test.workload]]
testName = 'BlobRestoreWorkload'
setupBlob = true
[[test]]
testTitle = 'BackupMutationLogs'
simBackupAgents = 'BackupToFile'
@ -32,9 +32,10 @@ waitForQuiescence = false
[[test.workload]]
testName = 'IncrementalBackup'
tag = 'default'
blobManifestUrl = 'file://simfdb/fdbblob/manifest'
submitOnly = true
waitForBackup = true
[[test]]
testTitle = 'WriteTest'
simBackupAgents = 'BackupToFile'