Merge pull request #2642 from xumengpanda/mengxu/new-backup-format-PR

FastRestore:Integrate with new backup format
This commit is contained in:
Evan Tschannen 2020-03-25 15:47:55 -07:00 committed by GitHub
commit bb5799bd20
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
31 changed files with 1691 additions and 829 deletions

View File

@ -68,7 +68,7 @@ void printLogFiles(std::string msg, const std::vector<LogFile>& files) {
std::vector<LogFile> getRelevantLogFiles(const std::vector<LogFile>& files, Version begin, Version end) {
std::vector<LogFile> filtered;
for (const auto& file : files) {
if (file.beginVersion <= end && file.endVersion >= begin && file.tagId >= 0) {
if (file.beginVersion <= end && file.endVersion >= begin && file.tagId >= 0 && file.fileSize > 0) {
filtered.push_back(file);
}
}
@ -76,15 +76,15 @@ std::vector<LogFile> getRelevantLogFiles(const std::vector<LogFile>& files, Vers
// Remove duplicates. This is because backup workers may store the log for
// old epochs successfully, but do not update the progress before another
// recovery happened. As a result, next epoch will retry and creates
// recovery happened. As a result, next epoch will retry and creates
// duplicated log files.
std::vector<LogFile> sorted;
int i = 0;
for (int j = 1; j < filtered.size(); j++) {
if (!filtered[i].sameContent(filtered[j])) {
if (!filtered[i].isSubset(filtered[j])) {
sorted.push_back(filtered[i]);
i = j;
}
i = j;
}
if (i < filtered.size()) {
sorted.push_back(filtered[i]);
@ -162,6 +162,9 @@ struct MutationFilesReadProgress : public ReferenceCounted<MutationFilesReadProg
Version msgVersion = invalidVersion;
try {
// Read block header
if (reader.consume<int32_t>() != PARTITIONED_MLOG_VERSION) throw restore_unsupported_file_version();
while (1) {
// If eof reached or first key len bytes is 0xFF then end of block was reached.
if (reader.eof() || *reader.rptr == 0xFF) break;

View File

@ -31,6 +31,7 @@ namespace file_converter {
enum {
OPT_CONTAINER,
OPT_BEGIN_VERSION,
OPT_CRASHONERROR,
OPT_END_VERSION,
OPT_TRACE,
OPT_TRACE_DIR,
@ -44,6 +45,7 @@ CSimpleOpt::SOption gConverterOptions[] = { { OPT_CONTAINER, "-r", SO_REQ_SEP },
{ OPT_CONTAINER, "--container", SO_REQ_SEP },
{ OPT_BEGIN_VERSION, "-b", SO_REQ_SEP },
{ OPT_BEGIN_VERSION, "--begin", SO_REQ_SEP },
{ OPT_CRASHONERROR, "--crash", SO_NONE },
{ OPT_END_VERSION, "-e", SO_REQ_SEP },
{ OPT_END_VERSION, "--end", SO_REQ_SEP },
{ OPT_TRACE, "--log", SO_NONE },
@ -59,7 +61,4 @@ CSimpleOpt::SOption gConverterOptions[] = { { OPT_CONTAINER, "-r", SO_REQ_SEP },
} // namespace file_converter
// Mutation log version written by old FileBackupAgent
static const uint32_t BACKUP_AGENT_MLOG_VERSION = 2001;
#endif // FDBBACKUP_FILECONVERTER_H

View File

@ -30,12 +30,17 @@
#include "flow/serialize.h"
#include "flow/actorcompiler.h" // has to be last include
#define SevDecodeInfo SevVerbose
extern bool g_crashOnError;
namespace file_converter {
void printDecodeUsage() {
std::cout << "\n"
" -r, --container Container URL.\n"
" -i, --input FILE Log file to be decoded.\n"
" --crash Crash on serious error.\n"
"\n";
return;
}
@ -89,6 +94,10 @@ int parseDecodeCommandLine(DecodeParams* param, CSimpleOpt* args) {
param->container_url = args->OptionArg();
break;
case OPT_CRASHONERROR:
g_crashOnError = true;
break;
case OPT_INPUT_FILE:
param->file = args->OptionArg();
break;
@ -161,7 +170,12 @@ std::vector<MutationRef> decode_value(const StringRef& value) {
reader.consume<uint64_t>(); // Consume the includeVersion
uint32_t val_length = reader.consume<uint32_t>();
ASSERT(val_length == value.size() - sizeof(uint64_t) - sizeof(uint32_t));
if (val_length != value.size() - sizeof(uint64_t) - sizeof(uint32_t)) {
TraceEvent(SevError, "ValueError")
.detail("ValueLen", val_length)
.detail("ValueSize", value.size())
.detail("Value", printable(value));
}
std::vector<MutationRef> mutations;
while (1) {
@ -217,54 +231,71 @@ struct DecodeProgress {
// The following are private APIs:
// Returns true if value contains complete data.
bool isValueComplete(StringRef value) {
StringRefReader reader(value, restore_corrupted_data());
reader.consume<uint64_t>(); // Consume the includeVersion
uint32_t val_length = reader.consume<uint32_t>();
return val_length == value.size() - sizeof(uint64_t) - sizeof(uint32_t);
}
// PRECONDITION: finished() must return false before calling this function.
// Returns the next batch of mutations along with the arena backing it.
ACTOR static Future<VersionedMutations> getNextBatchImpl(DecodeProgress* self) {
ASSERT(!self->finished());
state std::pair<Arena, KeyValueRef> arena_kv = self->keyValues[0];
// decode this batch's version
state std::pair<Version, int32_t> version_part = decode_key(arena_kv.second.key);
ASSERT(version_part.second == 0); // first part number must be 0.
// decode next versions, check if they are continuous parts
state int idx = 1; // next kv pair in "keyValues"
state int bufSize = arena_kv.second.value.size();
state int lastPart = 0;
loop {
// Try to decode another block if needed
if (idx == self->keyValues.size()) {
if (self->keyValues.size() == 1) {
// Try to decode another block when only one left
wait(readAndDecodeFile(self));
}
if (idx == self->keyValues.size()) break;
std::pair<Version, int32_t> next_version_part = decode_key(self->keyValues[idx].second.key);
if (version_part.first != next_version_part.first) break;
auto& tuple = self->keyValues[0];
ASSERT(std::get<2>(tuple) == 0); // first part number must be 0.
if (lastPart + 1 != next_version_part.second) {
TraceEvent("DecodeError").detail("Part1", lastPart).detail("Part2", next_version_part.second);
// decode next versions, check if they are continuous parts
int idx = 1; // next kv pair in "keyValues"
int bufSize = std::get<3>(tuple).size();
for (int lastPart = 0; idx < self->keyValues.size(); idx++, lastPart++) {
if (idx == self->keyValues.size()) break;
auto next_tuple = self->keyValues[idx];
if (std::get<1>(tuple) != std::get<1>(next_tuple)) {
break;
}
if (lastPart + 1 != std::get<2>(next_tuple)) {
TraceEvent("DecodeError").detail("Part1", lastPart).detail("Part2", std::get<2>(next_tuple));
throw restore_corrupted_data();
}
bufSize += std::get<3>(next_tuple).size();
}
VersionedMutations m;
m.version = std::get<1>(tuple);
TraceEvent("Decode").detail("Version", m.version).detail("Idx", idx).detail("Q", self->keyValues.size());
StringRef value = std::get<3>(tuple);
if (idx > 1) {
// Stitch parts into one and then decode one by one
Standalone<StringRef> buf = self->combineValues(idx, bufSize);
value = buf;
m.arena = buf.arena();
} else {
m.arena = std::get<0>(tuple);
}
if (self->isValueComplete(value)) {
m.mutations = decode_value(value);
self->keyValues.erase(self->keyValues.begin(), self->keyValues.begin() + idx);
return m;
} else if (!self->eof) {
// Read one more block, hopefully the missing part of the value can be found.
wait(readAndDecodeFile(self));
} else {
TraceEvent(SevError, "MissingValue").detail("Version", m.version);
throw restore_corrupted_data();
}
bufSize += self->keyValues[idx].second.value.size();
idx++;
lastPart++;
}
VersionedMutations m;
m.version = version_part.first;
if (idx > 1) {
// Stitch parts into one and then decode one by one
Standalone<StringRef> buf = self->combineValues(idx, bufSize);
m.mutations = decode_value(buf);
m.arena = buf.arena();
} else {
m.mutations = decode_value(arena_kv.second.value);
m.arena = arena_kv.first;
}
self->keyValues.erase(self->keyValues.begin(), self->keyValues.begin() + idx);
return m;
}
// Returns a buffer which stitches first "idx" values into one.
@ -275,7 +306,7 @@ struct DecodeProgress {
Standalone<StringRef> buf = makeString(len);
int n = 0;
for (int i = 0; i < idx; i++) {
const auto& value = keyValues[i].second.value;
const auto& value = std::get<3>(keyValues[i]);
memcpy(mutateString(buf) + n, value.begin(), value.size());
n += value.size();
}
@ -290,7 +321,7 @@ struct DecodeProgress {
StringRefReader reader(block, restore_corrupted_data());
try {
// Read header, currently only decoding version 2001
// Read header, currently only decoding version BACKUP_AGENT_MLOG_VERSION
if (reader.consume<int32_t>() != BACKUP_AGENT_MLOG_VERSION) throw restore_unsupported_file_version();
// Read k/v pairs. Block ends either at end of last value exactly or with 0xFF as first key len byte.
@ -301,9 +332,16 @@ struct DecodeProgress {
// Read key and value. If anything throws then there is a problem.
uint32_t kLen = reader.consumeNetworkUInt32();
const uint8_t* k = reader.consume(kLen);
std::pair<Version, int32_t> version_part = decode_key(StringRef(k, kLen));
uint32_t vLen = reader.consumeNetworkUInt32();
const uint8_t* v = reader.consume(vLen);
keyValues.emplace_back(buf.arena(), KeyValueRef(StringRef(k, kLen), StringRef(v, vLen)));
TraceEvent(SevDecodeInfo, "Block")
.detail("KeySize", kLen)
.detail("valueSize", vLen)
.detail("Offset", reader.rptr - buf.begin())
.detail("Version", version_part.first)
.detail("Part", version_part.second);
keyValues.emplace_back(buf.arena(), version_part.first, version_part.second, StringRef(v, vLen));
}
// Make sure any remaining bytes in the block are 0xFF
@ -311,6 +349,15 @@ struct DecodeProgress {
if (b != 0xFF) throw restore_corrupted_data_padding();
}
// The (version, part) in a block can be out of order, i.e., (3, 0)
// can be followed by (4, 0), and then (3, 1). So we need to sort them
// first by version, and then by part number.
std::sort(keyValues.begin(), keyValues.end(),
[](const std::tuple<Arena, Version, int32_t, StringRef>& a,
const std::tuple<Arena, Version, int32_t, StringRef>& b) {
return std::get<1>(a) == std::get<1>(b) ? std::get<2>(a) < std::get<2>(b)
: std::get<1>(a) < std::get<1>(b);
});
return;
} catch (Error& e) {
TraceEvent(SevWarn, "CorruptBlock").error(e).detail("Offset", reader.rptr - buf.begin());
@ -360,14 +407,21 @@ struct DecodeProgress {
Reference<IAsyncFile> fd;
int64_t offset = 0;
bool eof = false;
// Key value pairs and their memory arenas.
std::vector<std::pair<Arena, KeyValueRef>> keyValues;
// A (version, part_number)'s mutations and memory arena.
std::vector<std::tuple<Arena, Version, int32_t, StringRef>> keyValues;
};
ACTOR Future<Void> decode_logs(DecodeParams params) {
state Reference<IBackupContainer> container = IBackupContainer::openContainer(params.container_url);
state BackupFileList listing = wait(container->dumpFileList());
// remove partitioned logs
listing.logs.erase(std::remove_if(listing.logs.begin(), listing.logs.end(),
[](const LogFile& file) {
std::string prefix("plogs/");
return file.fileName.substr(0, prefix.size()) == prefix;
}),
listing.logs.end());
std::sort(listing.logs.begin(), listing.logs.end());
TraceEvent("Container").detail("URL", params.container_url).detail("Logs", listing.logs.size());

View File

@ -308,9 +308,16 @@ public:
/** BACKUP METHODS **/
Future<Void> submitBackup(Reference<ReadYourWritesTransaction> tr, Key outContainer, int snapshotIntervalSeconds, std::string tagName, Standalone<VectorRef<KeyRangeRef>> backupRanges, bool stopWhenDone = true);
Future<Void> submitBackup(Database cx, Key outContainer, int snapshotIntervalSeconds, std::string tagName, Standalone<VectorRef<KeyRangeRef>> backupRanges, bool stopWhenDone = true) {
return runRYWTransactionFailIfLocked(cx, [=](Reference<ReadYourWritesTransaction> tr){ return submitBackup(tr, outContainer, snapshotIntervalSeconds, tagName, backupRanges, stopWhenDone); });
Future<Void> submitBackup(Reference<ReadYourWritesTransaction> tr, Key outContainer, int snapshotIntervalSeconds,
std::string tagName, Standalone<VectorRef<KeyRangeRef>> backupRanges,
bool stopWhenDone = true, bool partitionedLog = false);
Future<Void> submitBackup(Database cx, Key outContainer, int snapshotIntervalSeconds, std::string tagName,
Standalone<VectorRef<KeyRangeRef>> backupRanges, bool stopWhenDone = true,
bool partitionedLog = false) {
return runRYWTransactionFailIfLocked(cx, [=](Reference<ReadYourWritesTransaction> tr) {
return submitBackup(tr, outContainer, snapshotIntervalSeconds, tagName, backupRanges, stopWhenDone,
partitionedLog);
});
}
Future<Void> discontinueBackup(Reference<ReadYourWritesTransaction> tr, Key tagName);
@ -787,6 +794,26 @@ public:
return configSpace.pack(LiteralStringRef(__FUNCTION__));
}
// Each backup worker adds its (epoch, tag.id) to this property.
KeyBackedProperty<std::vector<std::pair<int64_t, int64_t>>> startedBackupWorkers() {
return configSpace.pack(LiteralStringRef(__FUNCTION__));
}
// Set to true if backup worker is enabled.
KeyBackedProperty<bool> backupWorkerEnabled() {
return configSpace.pack(LiteralStringRef(__FUNCTION__));
}
// Set to true if partitioned log is enabled (only useful if backup worker is also enabled).
KeyBackedProperty<bool> partitionedLogEnabled() {
return configSpace.pack(LiteralStringRef(__FUNCTION__));
}
// Latest version for which all prior versions have saved by backup workers.
KeyBackedProperty<Version> latestBackupWorkerSavedVersion() {
return configSpace.pack(LiteralStringRef(__FUNCTION__));
}
// Stop differntial logging if already started or don't start after completing KV ranges
KeyBackedProperty<bool> stopWhenDone() {
return configSpace.pack(LiteralStringRef(__FUNCTION__));
@ -816,10 +843,17 @@ public:
tr->setOption(FDBTransactionOptions::READ_LOCK_AWARE);
auto lastLog = latestLogEndVersion().get(tr);
auto firstSnapshot = firstSnapshotEndVersion().get(tr);
return map(success(lastLog) && success(firstSnapshot), [=](Void) -> Optional<Version> {
auto workerEnabled = backupWorkerEnabled().get(tr);
auto plogEnabled = partitionedLogEnabled().get(tr);
auto workerVersion = latestBackupWorkerSavedVersion().get(tr);
return map(success(lastLog) && success(firstSnapshot) && success(workerEnabled) && success(plogEnabled) && success(workerVersion), [=](Void) -> Optional<Version> {
// The latest log greater than the oldest snapshot is the restorable version
if(lastLog.get().present() && firstSnapshot.get().present() && lastLog.get().get() > firstSnapshot.get().get()) {
return std::max(lastLog.get().get() - 1, firstSnapshot.get().get());
Optional<Version> logVersion = workerEnabled.get().present() && workerEnabled.get().get() &&
plogEnabled.get().present() && plogEnabled.get().get()
? workerVersion.get()
: lastLog.get();
if (logVersion.present() && firstSnapshot.get().present() && logVersion.get() > firstSnapshot.get().get()) {
return std::max(logVersion.get() - 1, firstSnapshot.get().get());
}
return {};
});
@ -880,6 +914,10 @@ struct StringRefReader {
const int32_t consumeNetworkInt32() { return (int32_t)bigEndian32((uint32_t)consume<int32_t>()); }
const uint32_t consumeNetworkUInt32() { return bigEndian32(consume<uint32_t>()); }
// Convert big Endian value (e.g., encoded in log file) into a littleEndian uint64_t value.
int64_t consumeNetworkInt64() { return (int64_t)bigEndian64((uint32_t)consume<int64_t>()); }
uint64_t consumeNetworkUInt64() { return bigEndian64(consume<uint64_t>()); }
bool eof() { return rptr == end; }
const uint8_t *rptr, *end;

View File

@ -228,28 +228,33 @@ std::string BackupDescription::toJSON() const {
* Snapshot manifests (a complete set of files constituting a database snapshot for the backup's target ranges)
* are stored as JSON files at paths like
* /snapshots/snapshot,minVersion,maxVersion,totalBytes
*
*
* Key range files for snapshots are stored at paths like
* /kvranges/snapshot,startVersion/N/range,version,uid,blockSize
* where startVersion is the version at which the backup snapshot execution began and N is a number
* that is increased as key range files are generated over time (at varying rates) such that there
* that is increased as key range files are generated over time (at varying rates) such that there
* are around 5,000 key range files in each folder.
*
* Note that startVersion will NOT correspond to the minVersion of a snapshot manifest because
* Note that startVersion will NOT correspond to the minVersion of a snapshot manifest because
* snapshot manifest min/max versions are based on the actual contained data and the first data
* file written will be after the start version of the snapshot's execution.
*
*
* Log files are at file paths like
* /logs/.../log,startVersion,endVersion,blockSize
* /plogs/...log,startVersion,endVersion,UID,tagID-of-N,blocksize
* /logs/.../log,startVersion,endVersion,UID,blockSize
* where ... is a multi level path which sorts lexically into version order and results in approximately 1
* unique folder per day containing about 5,000 files.
* unique folder per day containing about 5,000 files. Logs after 7.0 are stored in "plogs"
* directory and are partitioned according to tagIDs (0, 1, 2, ...) and the total number
* partitions is N. Logs before 7.0 are
* stored in "logs" directory and are not partitioned.
*
*
* BACKWARD COMPATIBILITY
*
* Prior to FDB version 6.0.16, key range files were stored using a different folder scheme. Newer versions
* still support this scheme for all restore and backup management operations but key range files generated
* by backup using version 6.0.16 or later use the scheme describe above.
*
* by backup using version 6.0.16 or later use the scheme describe above.
*
* The old format stored key range files at paths like
* /ranges/.../range,version,uid,blockSize
* where ... is a multi level path with sorts lexically into version order and results in up to approximately
@ -329,21 +334,22 @@ public:
}
// The innermost folder covers 100,000 seconds (1e11 versions) which is 5,000 mutation log files at current settings.
static std::string logVersionFolderString(Version v, bool mlogs) {
return format("%s/%s/", (mlogs ? "mlogs" : "logs"), versionFolderString(v, 11).c_str());
static std::string logVersionFolderString(Version v, bool partitioned) {
return format("%s/%s/", (partitioned ? "plogs" : "logs"), versionFolderString(v, 11).c_str());
}
Future<Reference<IBackupFile>> writeLogFile(Version beginVersion, Version endVersion, int blockSize) override {
Future<Reference<IBackupFile>> writeLogFile(Version beginVersion, Version endVersion, int blockSize) final {
return writeFile(logVersionFolderString(beginVersion, false) +
format("log,%lld,%lld,%s,%d", beginVersion, endVersion,
deterministicRandom()->randomUniqueID().toString().c_str(), blockSize));
}
Future<Reference<IBackupFile>> writeTaggedLogFile(Version beginVersion, Version endVersion, int blockSize,
uint16_t tagId) override {
uint16_t tagId, int totalTags) final {
return writeFile(logVersionFolderString(beginVersion, true) +
format("log,%lld,%lld,%s,%d,%d", beginVersion, endVersion,
deterministicRandom()->randomUniqueID().toString().c_str(), blockSize, tagId));
format("log,%lld,%lld,%s,%d-of-%d,%d", beginVersion, endVersion,
deterministicRandom()->randomUniqueID().toString().c_str(), tagId, totalTags,
blockSize));
}
Future<Reference<IBackupFile>> writeRangeFile(Version snapshotBeginVersion, int snapshotFileCount, Version fileVersion, int blockSize) override {
@ -394,8 +400,8 @@ public:
if(sscanf(name.c_str(), "log,%" SCNd64 ",%" SCNd64 ",%*[^,],%u%n", &f.beginVersion, &f.endVersion, &f.blockSize, &len) == 3 && len == name.size()) {
out = f;
return true;
} else if (sscanf(name.c_str(), "log,%" SCNd64 ",%" SCNd64 ",%*[^,],%u,%d%n", &f.beginVersion, &f.endVersion,
&f.blockSize, &f.tagId, &len) == 4 &&
} else if (sscanf(name.c_str(), "log,%" SCNd64 ",%" SCNd64 ",%*[^,],%d-of-%d,%u%n", &f.beginVersion,
&f.endVersion, &f.tagId, &f.totalTags, &f.blockSize, &len) == 5 &&
len == name.size() && f.tagId >= 0) {
out = f;
return true;
@ -484,7 +490,6 @@ public:
ACTOR static Future<Void> writeKeyspaceSnapshotFile_impl(Reference<BackupContainerFileSystem> bc, std::vector<std::string> fileNames, int64_t totalBytes) {
ASSERT(!fileNames.empty());
state Version minVer = std::numeric_limits<Version>::max();
state Version maxVer = 0;
state RangeFile rf;
@ -524,22 +529,23 @@ public:
return Void();
}
Future<Void> writeKeyspaceSnapshotFile(std::vector<std::string> fileNames, int64_t totalBytes) override {
Future<Void> writeKeyspaceSnapshotFile(std::vector<std::string> fileNames, int64_t totalBytes) final {
return writeKeyspaceSnapshotFile_impl(Reference<BackupContainerFileSystem>::addRef(this), fileNames, totalBytes);
};
// List log files, unsorted, which contain data at any version >= beginVersion and <= targetVersion
Future<std::vector<LogFile>> listLogFiles(Version beginVersion = 0, Version targetVersion = std::numeric_limits<Version>::max()) {
// The first relevant log file could have a begin version less than beginVersion based on the knobs which determine log file range size,
// so start at an earlier version adjusted by how many versions a file could contain.
// List log files, unsorted, which contain data at any version >= beginVersion and <= targetVersion.
// "partitioned" flag indicates if new partitioned mutation logs or old logs should be listed.
Future<std::vector<LogFile>> listLogFiles(Version beginVersion, Version targetVersion, bool partitioned) {
// The first relevant log file could have a begin version less than beginVersion based on the knobs which
// determine log file range size, so start at an earlier version adjusted by how many versions a file could
// contain.
//
// Get the cleaned (without slashes) first and last folders that could contain relevant results.
bool mlogs = false; // tagged mutation logs
std::string firstPath = cleanFolderString(
logVersionFolderString(std::max<Version>(0, beginVersion - CLIENT_KNOBS->BACKUP_MAX_LOG_RANGES *
CLIENT_KNOBS->LOG_RANGE_BLOCK_SIZE),
mlogs));
std::string lastPath = cleanFolderString(logVersionFolderString(targetVersion, mlogs));
partitioned));
std::string lastPath = cleanFolderString(logVersionFolderString(targetVersion, partitioned));
std::function<bool(std::string const &)> pathFilter = [=](const std::string &folderPath) {
// Remove slashes in the given folder path so that the '/' positions in the version folder string do not matter
@ -549,7 +555,7 @@ public:
|| (cleaned > firstPath && cleaned < lastPath);
};
return map(listFiles("logs/", pathFilter), [=](const FilesAndSizesT &files) {
return map(listFiles((partitioned ? "plogs/" : "logs/"), pathFilter), [=](const FilesAndSizesT& files) {
std::vector<LogFile> results;
LogFile lf;
for(auto &f : files) {
@ -636,11 +642,15 @@ public:
ACTOR static Future<BackupFileList> dumpFileList_impl(Reference<BackupContainerFileSystem> bc, Version begin, Version end) {
state Future<std::vector<RangeFile>> fRanges = bc->listRangeFiles(begin, end);
state Future<std::vector<KeyspaceSnapshotFile>> fSnapshots = bc->listKeyspaceSnapshots(begin, end);
state Future<std::vector<LogFile>> fLogs = bc->listLogFiles(begin, end);
state std::vector<LogFile> logs;
state std::vector<LogFile> pLogs;
wait(success(fRanges) && success(fSnapshots) && success(fLogs));
wait(success(fRanges) && success(fSnapshots) &&
store(logs, bc->listLogFiles(begin, end, false)) &&
store(pLogs, bc->listLogFiles(begin, end, true)));
logs.insert(logs.end(), std::make_move_iterator(pLogs.begin()), std::make_move_iterator(pLogs.end()));
return BackupFileList({fRanges.get(), fLogs.get(), fSnapshots.get()});
return BackupFileList({ fRanges.get(), std::move(logs), fSnapshots.get() });
}
Future<BackupFileList> dumpFileList(Version begin, Version end) override {
@ -662,7 +672,27 @@ public:
return v;
}
ACTOR static Future<BackupDescription> describeBackup_impl(Reference<BackupContainerFileSystem> bc, bool deepScan, Version logStartVersionOverride) {
// Computes the continuous end version for non-partitioned mutation logs up to
// the "targetVersion". If "outLogs" is not nullptr, it will be updated with
// continuous log files. "*end" is updated with the continuous end version.
static void computeRestoreEndVersion(const std::vector<LogFile>& logs, std::vector<LogFile>* outLogs, Version* end,
Version targetVersion) {
auto i = logs.begin();
if (outLogs != nullptr) outLogs->push_back(*i);
// Add logs to restorable logs set until continuity is broken OR we reach targetVersion
while (++i != logs.end()) {
if (i->beginVersion > *end || i->beginVersion > targetVersion) break;
// If the next link in the log chain is found, update the end
if (i->beginVersion == *end) {
if (outLogs != nullptr) outLogs->push_back(*i);
*end = i->endVersion;
}
}
}
ACTOR static Future<BackupDescription> describeBackup_impl(Reference<BackupContainerFileSystem> bc, bool deepScan, Version logStartVersionOverride, bool partitioned) {
state BackupDescription desc;
desc.url = bc->getURL();
@ -680,8 +710,10 @@ public:
// from which to resolve the relative version.
// This could be handled more efficiently without recursion but it's tricky, this will do for now.
if(logStartVersionOverride != invalidVersion && logStartVersionOverride < 0) {
BackupDescription tmp = wait(bc->describeBackup(false, invalidVersion));
logStartVersionOverride = resolveRelativeVersion(tmp.maxLogEnd, logStartVersionOverride, "LogStartVersionOverride", invalid_option_value());
BackupDescription tmp = wait(partitioned ? bc->describePartitionedBackup(false, invalidVersion)
: bc->describeBackup(false, invalidVersion));
logStartVersionOverride = resolveRelativeVersion(tmp.maxLogEnd, logStartVersionOverride,
"LogStartVersionOverride", invalid_option_value());
}
// Get metadata versions
@ -767,31 +799,26 @@ public:
}
state std::vector<LogFile> logs;
wait(store(logs, bc->listLogFiles(scanBegin, scanEnd)) && store(desc.snapshots, bc->listKeyspaceSnapshots()));
wait(store(logs, bc->listLogFiles(scanBegin, scanEnd, partitioned)) &&
store(desc.snapshots, bc->listKeyspaceSnapshots()));
// List logs in version order so log continuity can be analyzed
std::sort(logs.begin(), logs.end());
if(!logs.empty()) {
// Find out contiguous log end version
if (!logs.empty()) {
desc.maxLogEnd = logs.rbegin()->endVersion;
auto i = logs.begin();
// If we didn't get log versions above then seed them using the first log file
if(!desc.contiguousLogEnd.present()) {
desc.minLogBegin = i->beginVersion;
desc.contiguousLogEnd = i->endVersion;
++i;
if (!desc.contiguousLogEnd.present()) {
desc.minLogBegin = logs.begin()->beginVersion;
desc.contiguousLogEnd = logs.begin()->endVersion;
}
auto &end = desc.contiguousLogEnd.get(); // For convenience to make loop cleaner
// Advance until continuity is broken
while(i != logs.end()) {
if(i->beginVersion > end)
break;
// If the next link in the log chain is found, update the end
if(i->beginVersion == end)
end = i->endVersion;
++i;
if (partitioned) {
determinePartitionedLogsBeginEnd(&desc, logs);
} else {
Version& end = desc.contiguousLogEnd.get();
computeRestoreEndVersion(logs, nullptr, &end, std::numeric_limits<Version>::max());
}
}
@ -860,8 +887,12 @@ public:
}
// Uses the virtual methods to describe the backup contents
Future<BackupDescription> describeBackup(bool deepScan, Version logStartVersionOverride) override {
return describeBackup_impl(Reference<BackupContainerFileSystem>::addRef(this), deepScan, logStartVersionOverride);
Future<BackupDescription> describeBackup(bool deepScan, Version logStartVersionOverride) final {
return describeBackup_impl(Reference<BackupContainerFileSystem>::addRef(this), deepScan, logStartVersionOverride, false);
}
Future<BackupDescription> describePartitionedBackup(bool deepScan, Version logStartVersionOverride) final {
return describeBackup_impl(Reference<BackupContainerFileSystem>::addRef(this), deepScan, logStartVersionOverride, true);
}
ACTOR static Future<Void> expireData_impl(Reference<BackupContainerFileSystem> bc, Version expireEndVersion, bool force, ExpireProgress *progress, Version restorableBeginVersion) {
@ -879,8 +910,10 @@ public:
state BackupDescription desc = wait(bc->describeBackup(false, expireEndVersion));
// Resolve relative versions using max log version
expireEndVersion = resolveRelativeVersion(desc.maxLogEnd, expireEndVersion, "ExpireEndVersion", invalid_option_value());
restorableBeginVersion = resolveRelativeVersion(desc.maxLogEnd, restorableBeginVersion, "RestorableBeginVersion", invalid_option_value());
expireEndVersion =
resolveRelativeVersion(desc.maxLogEnd, expireEndVersion, "ExpireEndVersion", invalid_option_value());
restorableBeginVersion = resolveRelativeVersion(desc.maxLogEnd, restorableBeginVersion,
"RestorableBeginVersion", invalid_option_value());
// It would be impossible to have restorability to any version < expireEndVersion after expiring to that version
if(restorableBeginVersion < expireEndVersion)
@ -921,13 +954,17 @@ public:
.detail("ScanBeginVersion", scanBegin);
state std::vector<LogFile> logs;
state std::vector<LogFile> pLogs; // partitioned mutation logs
state std::vector<RangeFile> ranges;
if(progress != nullptr) {
progress->step = "Listing files";
}
// Get log files or range files that contain any data at or before expireEndVersion
wait(store(logs, bc->listLogFiles(scanBegin, expireEndVersion - 1)) && store(ranges, bc->listRangeFiles(scanBegin, expireEndVersion - 1)));
wait(store(logs, bc->listLogFiles(scanBegin, expireEndVersion - 1, false)) &&
store(pLogs, bc->listLogFiles(scanBegin, expireEndVersion - 1, true)) &&
store(ranges, bc->listRangeFiles(scanBegin, expireEndVersion - 1)));
logs.insert(logs.end(), std::make_move_iterator(pLogs.begin()), std::make_move_iterator(pLogs.end()));
// The new logBeginVersion will be taken from the last log file, if there is one
state Optional<Version> newLogBeginVersion;
@ -1040,11 +1077,167 @@ public:
}
// Delete all data up to (but not including endVersion)
Future<Void> expireData(Version expireEndVersion, bool force, ExpireProgress *progress, Version restorableBeginVersion) override {
Future<Void> expireData(Version expireEndVersion, bool force, ExpireProgress* progress,
Version restorableBeginVersion) final {
return expireData_impl(Reference<BackupContainerFileSystem>::addRef(this), expireEndVersion, force, progress, restorableBeginVersion);
}
ACTOR static Future<Optional<RestorableFileSet>> getRestoreSet_impl(Reference<BackupContainerFileSystem> bc, Version targetVersion) {
// For a list of log files specified by their indices (of the same tag),
// returns if they are continous in the range [begin, end]. If "tags" is not
// nullptr, then it will be populated with [begin, end] -> tags, where next
// pair's begin <= previous pair's end + 1. On return, the last pair's end
// version (inclusive) gives the continuous range from begin.
static bool isContinuous(const std::vector<LogFile>& files, const std::vector<int>& indices, Version begin,
Version end, std::map<std::pair<Version, Version>, int>* tags) {
Version lastBegin = invalidVersion;
Version lastEnd = invalidVersion;
int lastTags = -1;
ASSERT(tags == nullptr || tags->empty());
for (int idx : indices) {
const LogFile& file = files[idx];
if (lastEnd == invalidVersion) {
if (file.beginVersion > begin) return false;
if (file.endVersion > begin) {
lastBegin = begin;
lastTags = file.totalTags;
} else {
continue;
}
} else if (lastEnd < file.beginVersion) {
if (tags != nullptr) {
tags->emplace(std::make_pair(lastBegin, lastEnd - 1), lastTags);
}
return false;
}
if (lastTags != file.totalTags) {
if (tags != nullptr) {
tags->emplace(std::make_pair(lastBegin, file.beginVersion - 1), lastTags);
}
lastBegin = file.beginVersion;
lastTags = file.totalTags;
}
lastEnd = file.endVersion;
if (lastEnd > end) break;
}
if (tags != nullptr && lastBegin != invalidVersion) {
tags->emplace(std::make_pair(lastBegin, std::min(end, lastEnd - 1)), lastTags);
}
return lastBegin != invalidVersion && lastEnd > end;
}
// Returns true if logs are continuous in the range [begin, end].
// "files" should be pre-sorted according to version order.
static bool isPartitionedLogsContinuous(const std::vector<LogFile>& files, Version begin, Version end) {
std::map<int, std::vector<int>> tagIndices; // tagId -> indices in files
for (int i = 0; i < files.size(); i++) {
ASSERT(files[i].tagId >= 0 && files[i].tagId < files[i].totalTags);
auto& indices = tagIndices[files[i].tagId];
indices.push_back(i);
}
// check tag 0 is continuous and create a map of ranges to tags
std::map<std::pair<Version, Version>, int> tags; // range [start, end] -> tags
if (!isContinuous(files, tagIndices[0], begin, end, &tags)) return false;
// for each range in tags, check all tags from 1 are continouous
for (const auto [beginEnd, count] : tags) {
for (int i = 1; i < count; i++) {
if (!isContinuous(files, tagIndices[i], beginEnd.first, beginEnd.second, nullptr)) {
return false;
}
}
}
return true;
}
// Returns log files that are not duplicated, or subset of another log.
// If a log file's progress is not saved, a new log file will be generated
// with the same begin version. So we can have a file that contains a subset
// of contents in another log file.
// PRE-CONDITION: logs are already sorted by (tagId, beginVersion, endVersion).
static std::vector<LogFile> filterDuplicates(const std::vector<LogFile>& logs) {
std::vector<LogFile> filtered;
int i = 0;
for (int j = 1; j < logs.size(); j++) {
if (logs[j].isSubset(logs[i])) continue;
if (!logs[i].isSubset(logs[j])) {
filtered.push_back(logs[i]);
}
i = j;
}
if (i < logs.size()) filtered.push_back(logs[i]);
return filtered;
}
// Analyze partitioned logs and set minLogBegin and contiguousLogEnd.
// For partitioned logs, different tags may start at different versions, so
// we need to find the "minLogBegin" version as well.
static void determinePartitionedLogsBeginEnd(BackupDescription* desc, const std::vector<LogFile>& logs) {
if (logs.empty()) return;
for (const LogFile& file : logs) {
Version end = getPartitionedLogsContinuousEndVersion(logs, file.beginVersion);
if (end > file.beginVersion) {
// desc->minLogBegin = file.beginVersion;
// contiguousLogEnd is not inclusive, so +1 here.
desc->contiguousLogEnd.get() = end + 1;
return;
}
}
}
// Returns the end version such that [begin, end] is continuous.
// "logs" should be already sorted.
static Version getPartitionedLogsContinuousEndVersion(const std::vector<LogFile>& logs, Version begin) {
Version end = 0;
std::map<int, std::vector<int>> tagIndices; // tagId -> indices in files
for (int i = 0; i < logs.size(); i++) {
ASSERT(logs[i].tagId >= 0 && logs[i].tagId < logs[i].totalTags);
auto& indices = tagIndices[logs[i].tagId];
// filter out if indices.back() is subset of files[i] or vice versa
if (!indices.empty()) {
if (logs[indices.back()].isSubset(logs[i])) {
indices.back() = i;
} else if (!logs[i].isSubset(logs[indices.back()])) {
indices.push_back(i);
}
} else {
indices.push_back(i);
}
end = std::max(end, logs[i].endVersion - 1);
}
// check tag 0 is continuous in [begin, end] and create a map of ranges to tags
std::map<std::pair<Version, Version>, int> tags; // range [start, end] -> tags
isContinuous(logs, tagIndices[0], begin, end, &tags);
if (tags.empty() || end <= begin) return 0;
end = std::min(end, tags.rbegin()->first.second);
// for each range in tags, check all tags from 1 are continouous
Version lastEnd = begin;
for (const auto [beginEnd, count] : tags) {
Version tagEnd = end; // This range's minimum continous tag version
for (int i = 1; i < count; i++) {
std::map<std::pair<Version, Version>, int> rangeTags;
isContinuous(logs, tagIndices[i], beginEnd.first, beginEnd.second, &rangeTags);
tagEnd = rangeTags.empty() ? 0 : std::min(tagEnd, rangeTags.rbegin()->first.second);
if (tagEnd == 0) return lastEnd;
}
if (tagEnd < beginEnd.second) {
end = tagEnd;
break;
}
lastEnd = beginEnd.second;
}
return end;
}
ACTOR static Future<Optional<RestorableFileSet>> getRestoreSet_impl(Reference<BackupContainerFileSystem> bc, Version targetVersion, bool partitioned) {
// Find the most recent keyrange snapshot to end at or before targetVersion
state Optional<KeyspaceSnapshotFile> snapshot;
std::vector<KeyspaceSnapshotFile> snapshots = wait(bc->listKeyspaceSnapshots());
@ -1067,29 +1260,36 @@ public:
return Optional<RestorableFileSet>(restorable);
}
state std::vector<LogFile> logs = wait(bc->listLogFiles(snapshot.get().beginVersion, targetVersion));
// FIXME: check if there are tagged logs. for each tag, there is no version gap.
state std::vector<LogFile> logs = wait(bc->listLogFiles(snapshot.get().beginVersion, targetVersion, partitioned));
if (partitioned) {
// sort by tag ID so that filterDuplicates works.
std::sort(logs.begin(), logs.end(), [](const LogFile& a, const LogFile& b) {
return std::tie(a.tagId, a.beginVersion, a.endVersion) <
std::tie(b.tagId, b.beginVersion, b.endVersion);
});
// Remove duplicated log files that can happen for old epochs.
std::vector<LogFile> filtered = filterDuplicates(logs);
restorable.logs.swap(filtered);
// sort by version order again for continuous analysis
std::sort(restorable.logs.begin(), restorable.logs.end());
if (isPartitionedLogsContinuous(restorable.logs, snapshot.get().beginVersion, targetVersion)) {
return Optional<RestorableFileSet>(restorable);
}
return Optional<RestorableFileSet>();
}
// List logs in version order so log continuity can be analyzed
std::sort(logs.begin(), logs.end());
// If there are logs and the first one starts at or before the snapshot begin version then proceed
if(!logs.empty() && logs.front().beginVersion <= snapshot.get().beginVersion) {
auto i = logs.begin();
Version end = i->endVersion;
restorable.logs.push_back(*i);
// Add logs to restorable logs set until continuity is broken OR we reach targetVersion
while(++i != logs.end()) {
if(i->beginVersion > end || i->beginVersion > targetVersion)
break;
// If the next link in the log chain is found, update the end
if(i->beginVersion == end) {
restorable.logs.push_back(*i);
end = i->endVersion;
}
}
if(end >= targetVersion) {
Version end = logs.begin()->endVersion;
computeRestoreEndVersion(logs, &restorable.logs, &end, targetVersion);
if (end >= targetVersion) {
return Optional<RestorableFileSet>(restorable);
}
}
@ -1098,8 +1298,12 @@ public:
return Optional<RestorableFileSet>();
}
Future<Optional<RestorableFileSet>> getRestoreSet(Version targetVersion) override {
return getRestoreSet_impl(Reference<BackupContainerFileSystem>::addRef(this), targetVersion);
Future<Optional<RestorableFileSet>> getRestoreSet(Version targetVersion) final {
return getRestoreSet_impl(Reference<BackupContainerFileSystem>::addRef(this), targetVersion, false);
}
Future<Optional<RestorableFileSet>> getPartitionedRestoreSet(Version targetVersion) final {
return getRestoreSet_impl(Reference<BackupContainerFileSystem>::addRef(this), targetVersion, true);
}
private:
@ -1183,8 +1387,8 @@ public:
class BackupContainerLocalDirectory : public BackupContainerFileSystem, ReferenceCounted<BackupContainerLocalDirectory> {
public:
void addref() override { return ReferenceCounted<BackupContainerLocalDirectory>::addref(); }
void delref() override { return ReferenceCounted<BackupContainerLocalDirectory>::delref(); }
void addref() final { return ReferenceCounted<BackupContainerLocalDirectory>::addref(); }
void delref() final { return ReferenceCounted<BackupContainerLocalDirectory>::delref(); }
static std::string getURLFormat() { return "file://</path/to/base/dir/>"; }
@ -1233,7 +1437,7 @@ public:
return results;
}
Future<Void> create() override {
Future<Void> create() final {
// Nothing should be done here because create() can be called by any process working with the container URL, such as fdbbackup.
// Since "local directory" containers are by definition local to the machine they are accessed from,
// the container's creation (in this case the creation of a directory) must be ensured prior to every file creation,
@ -1243,11 +1447,11 @@ public:
}
// The container exists if the folder it resides in exists
Future<bool> exists() override {
Future<bool> exists() final {
return directoryExists(m_path);
}
Future<Reference<IAsyncFile>> readFile(std::string path) override {
Future<Reference<IAsyncFile>> readFile(std::string path) final {
int flags = IAsyncFile::OPEN_NO_AIO | IAsyncFile::OPEN_READONLY | IAsyncFile::OPEN_UNCACHED;
// Simulation does not properly handle opening the same file from multiple machines using a shared filesystem,
// so create a symbolic link to make each file opening appear to be unique. This could also work in production
@ -1272,15 +1476,16 @@ public:
int blockSize = 0;
// Extract block size from the filename, if present
size_t lastComma = path.find_last_of(',');
if(lastComma != path.npos) {
if (lastComma != path.npos) {
blockSize = atoi(path.substr(lastComma + 1).c_str());
}
if(blockSize <= 0) {
if (blockSize <= 0) {
blockSize = deterministicRandom()->randomInt(1e4, 1e6);
}
if(deterministicRandom()->random01() < .01) {
blockSize /= deterministicRandom()->randomInt(1, 3);
}
ASSERT(blockSize > 0);
return map(f, [=](Reference<IAsyncFile> fr) {
int readAhead = deterministicRandom()->randomInt(0, 3);
@ -1324,7 +1529,7 @@ public:
std::string m_finalFullPath;
};
Future<Reference<IBackupFile>> writeFile(std::string path) override {
Future<Reference<IBackupFile>> writeFile(std::string path) final {
int flags = IAsyncFile::OPEN_NO_AIO | IAsyncFile::OPEN_CREATE | IAsyncFile::OPEN_ATOMIC_WRITE_AND_CREATE | IAsyncFile::OPEN_READWRITE;
std::string fullPath = joinPath(m_path, path);
platform::createDirectory(parentDirectory(fullPath));
@ -1335,12 +1540,12 @@ public:
});
}
Future<Void> deleteFile(std::string path) override {
Future<Void> deleteFile(std::string path) final {
::deleteFile(joinPath(m_path, path));
return Void();
}
Future<FilesAndSizesT> listFiles(std::string path, std::function<bool(std::string const &)>) {
Future<FilesAndSizesT> listFiles(std::string path, std::function<bool(std::string const&)>) final {
FilesAndSizesT results;
std::vector<std::string> files;
@ -1360,7 +1565,7 @@ public:
return results;
}
Future<Void> deleteContainer(int* pNumDeleted) override {
Future<Void> deleteContainer(int* pNumDeleted) final {
// In order to avoid deleting some random directory due to user error, first describe the backup
// and make sure it has something in it.
return map(describeBackup(false, invalidVersion), [=](BackupDescription const &desc) {
@ -1420,8 +1625,8 @@ public:
}
}
void addref() override { return ReferenceCounted<BackupContainerBlobStore>::addref(); }
void delref() override { return ReferenceCounted<BackupContainerBlobStore>::delref(); }
void addref() final { return ReferenceCounted<BackupContainerBlobStore>::addref(); }
void delref() final { return ReferenceCounted<BackupContainerBlobStore>::delref(); }
static std::string getURLFormat() {
return BlobStoreEndpoint::getURLFormat(true) + " (Note: The 'bucket' parameter is required.)";
@ -1429,16 +1634,17 @@ public:
virtual ~BackupContainerBlobStore() {}
Future<Reference<IAsyncFile>> readFile(std::string path) override {
return Reference<IAsyncFile>(
new AsyncFileReadAheadCache(
Reference<IAsyncFile>(new AsyncFileBlobStoreRead(m_bstore, m_bucket, dataPath(path))),
m_bstore->knobs.read_block_size,
m_bstore->knobs.read_ahead_blocks,
m_bstore->knobs.concurrent_reads_per_file,
m_bstore->knobs.read_cache_blocks_per_file
)
);
Future<Reference<IAsyncFile>> readFile(std::string path) final {
ASSERT(m_bstore->knobs.read_ahead_blocks > 0);
return Reference<IAsyncFile>(
new AsyncFileReadAheadCache(
Reference<IAsyncFile>(new AsyncFileBlobStoreRead(m_bstore, m_bucket, dataPath(path))),
m_bstore->knobs.read_block_size,
m_bstore->knobs.read_ahead_blocks,
m_bstore->knobs.concurrent_reads_per_file,
m_bstore->knobs.read_cache_blocks_per_file
)
);
}
ACTOR static Future<std::vector<std::string>> listURLs(Reference<BlobStoreEndpoint> bstore, std::string bucket) {
@ -1466,17 +1672,18 @@ public:
return map(m_file->sync(), [=](Void _) { self->m_file.clear(); return Void(); });
}
void addref() override { return ReferenceCounted<BackupFile>::addref(); }
void delref() override { return ReferenceCounted<BackupFile>::delref(); }
void addref() final { return ReferenceCounted<BackupFile>::addref(); }
void delref() final { return ReferenceCounted<BackupFile>::delref(); }
private:
Reference<IAsyncFile> m_file;
};
Future<Reference<IBackupFile>> writeFile(std::string path) override {
Future<Reference<IBackupFile>> writeFile(std::string path) final {
return Reference<IBackupFile>(new BackupFile(path, Reference<IAsyncFile>(new AsyncFileBlobStoreWrite(m_bstore, m_bucket, dataPath(path)))));
}
Future<Void> deleteFile(std::string path) override {
Future<Void> deleteFile(std::string path) final {
return m_bstore->deleteObject(m_bucket, dataPath(path));
}
@ -1498,7 +1705,7 @@ public:
return files;
}
Future<FilesAndSizesT> listFiles(std::string path, std::function<bool(std::string const &)> pathFilter) {
Future<FilesAndSizesT> listFiles(std::string path, std::function<bool(std::string const &)> pathFilter) final {
return listFiles_impl(Reference<BackupContainerBlobStore>::addRef(this), path, pathFilter);
}
@ -1514,12 +1721,12 @@ public:
return Void();
}
Future<Void> create() override {
Future<Void> create() final {
return create_impl(Reference<BackupContainerBlobStore>::addRef(this));
}
// The container exists if the index entry in the blob bucket exists
Future<bool> exists() override {
Future<bool> exists() final {
return m_bstore->objectExists(m_bucket, indexEntry());
}
@ -1539,7 +1746,7 @@ public:
return Void();
}
Future<Void> deleteContainer(int* pNumDeleted) override {
Future<Void> deleteContainer(int* pNumDeleted) final {
return deleteContainer_impl(Reference<BackupContainerBlobStore>::addRef(this), pNumDeleted);
}
@ -1963,3 +2170,65 @@ TEST_CASE("/backup/time") {
return Void();
}
TEST_CASE("/backup/continuous") {
std::vector<LogFile> files;
// [0, 100) 2 tags
files.push_back({ 0, 100, 10, "file1", 100, 0, 2 }); // Tag 0: 0-100
ASSERT(!BackupContainerFileSystem::isPartitionedLogsContinuous(files, 0, 99));
ASSERT(BackupContainerFileSystem::getPartitionedLogsContinuousEndVersion(files, 0) == 0);
files.push_back({ 0, 100, 10, "file2", 200, 1, 2 }); // Tag 1: 0-100
std::sort(files.begin(), files.end());
ASSERT(BackupContainerFileSystem::isPartitionedLogsContinuous(files, 0, 99));
ASSERT(!BackupContainerFileSystem::isPartitionedLogsContinuous(files, 0, 100));
ASSERT(BackupContainerFileSystem::getPartitionedLogsContinuousEndVersion(files, 0) == 99);
// [100, 300) 3 tags
files.push_back({ 100, 200, 10, "file3", 200, 0, 3 }); // Tag 0: 100-200
files.push_back({ 100, 250, 10, "file4", 200, 1, 3 }); // Tag 1: 100-250
std::sort(files.begin(), files.end());
ASSERT(BackupContainerFileSystem::isPartitionedLogsContinuous(files, 0, 99));
ASSERT(!BackupContainerFileSystem::isPartitionedLogsContinuous(files, 0, 100));
ASSERT(!BackupContainerFileSystem::isPartitionedLogsContinuous(files, 50, 150));
ASSERT(BackupContainerFileSystem::getPartitionedLogsContinuousEndVersion(files, 0) == 99);
files.push_back({ 100, 300, 10, "file5", 200, 2, 3 }); // Tag 2: 100-300
std::sort(files.begin(), files.end());
ASSERT(BackupContainerFileSystem::isPartitionedLogsContinuous(files, 50, 150));
ASSERT(!BackupContainerFileSystem::isPartitionedLogsContinuous(files, 50, 200));
ASSERT(BackupContainerFileSystem::isPartitionedLogsContinuous(files, 10, 199));
ASSERT(BackupContainerFileSystem::getPartitionedLogsContinuousEndVersion(files, 0) == 199);
ASSERT(BackupContainerFileSystem::getPartitionedLogsContinuousEndVersion(files, 100) == 199);
files.push_back({ 250, 300, 10, "file6", 200, 0, 3 }); // Tag 0: 250-300, missing 200-250
std::sort(files.begin(), files.end());
ASSERT(!BackupContainerFileSystem::isPartitionedLogsContinuous(files, 50, 240));
ASSERT(!BackupContainerFileSystem::isPartitionedLogsContinuous(files, 100, 280));
ASSERT(BackupContainerFileSystem::getPartitionedLogsContinuousEndVersion(files, 99) == 199);
files.push_back({ 250, 300, 10, "file7", 200, 1, 3 }); // Tag 1: 250-300
std::sort(files.begin(), files.end());
ASSERT(!BackupContainerFileSystem::isPartitionedLogsContinuous(files, 100, 280));
files.push_back({ 200, 250, 10, "file8", 200, 0, 3 }); // Tag 0: 200-250
std::sort(files.begin(), files.end());
ASSERT(BackupContainerFileSystem::isPartitionedLogsContinuous(files, 0, 299));
ASSERT(BackupContainerFileSystem::isPartitionedLogsContinuous(files, 100, 280));
ASSERT(BackupContainerFileSystem::getPartitionedLogsContinuousEndVersion(files, 150) == 299);
// [300, 400) 1 tag
// files.push_back({200, 250, 10, "file9", 200, 0, 3}); // Tag 0: 200-250, duplicate file
files.push_back({ 300, 400, 10, "file10", 200, 0, 1 }); // Tag 1: 300-400
std::sort(files.begin(), files.end());
ASSERT(BackupContainerFileSystem::isPartitionedLogsContinuous(files, 0, 399));
ASSERT(BackupContainerFileSystem::isPartitionedLogsContinuous(files, 100, 399));
ASSERT(BackupContainerFileSystem::isPartitionedLogsContinuous(files, 150, 399));
ASSERT(BackupContainerFileSystem::isPartitionedLogsContinuous(files, 250, 399));
ASSERT(BackupContainerFileSystem::getPartitionedLogsContinuousEndVersion(files, 0) == 399);
ASSERT(BackupContainerFileSystem::getPartitionedLogsContinuousEndVersion(files, 99) == 399);
ASSERT(BackupContainerFileSystem::getPartitionedLogsContinuousEndVersion(files, 250) == 399);
return Void();
}

View File

@ -62,6 +62,12 @@ protected:
// Structures for various backup components
// Mutation log version written by old FileBackupAgent
static const uint32_t BACKUP_AGENT_MLOG_VERSION = 2001;
// Mutation log version written by BackupWorker
static const uint32_t PARTITIONED_MLOG_VERSION = 4110;
struct LogFile {
Version beginVersion;
Version endVersion;
@ -69,15 +75,17 @@ struct LogFile {
std::string fileName;
int64_t fileSize;
int tagId = -1; // Log router tag. Non-negative for new backup format.
int totalTags = -1; // Total number of log router tags.
// Order by beginVersion, break ties with endVersion
bool operator< (const LogFile &rhs) const {
return beginVersion == rhs.beginVersion ? endVersion < rhs.endVersion : beginVersion < rhs.beginVersion;
}
// Returns if two log files have the same content by comparing version range and tag ID.
bool sameContent(const LogFile& rhs) const {
return beginVersion == rhs.beginVersion && endVersion == rhs.endVersion && tagId == rhs.tagId;
// Returns if this log file contains a subset of content of the given file
// by comparing version range and tag ID.
bool isSubset(const LogFile& rhs) const {
return beginVersion >= rhs.beginVersion && endVersion <= rhs.endVersion && tagId == rhs.tagId;
}
std::string toString() const {
@ -214,7 +222,7 @@ public:
// Open a tagged log file for writing, where tagId is the log router tag's id.
virtual Future<Reference<IBackupFile>> writeTaggedLogFile(Version beginVersion, Version endVersion, int blockSize,
uint16_t tagId) = 0;
uint16_t tagId, int totalTags) = 0;
// Write a KeyspaceSnapshotFile of range file names representing a full non overlapping
// snapshot of the key ranges this backup is targeting.
@ -248,12 +256,20 @@ public:
// be after deleting all data prior to logStartVersionOverride.
virtual Future<BackupDescription> describeBackup(bool deepScan = false, Version logStartVersionOverride = invalidVersion) = 0;
// The same as above, except using partitioned mutation logs.
virtual Future<BackupDescription> describePartitionedBackup(bool deepScan = false, Version logStartVersionOverride = invalidVersion) = 0;
virtual Future<BackupFileList> dumpFileList(Version begin = 0, Version end = std::numeric_limits<Version>::max()) = 0;
// Get exactly the files necessary to restore to targetVersion. Returns non-present if
// restore to given version is not possible.
virtual Future<Optional<RestorableFileSet>> getRestoreSet(Version targetVersion) = 0;
// Get exactly the files necessary to restore to targetVersion. Returns non-present if
// restore to given version is not possible. This is intended for parallel
// restore in FDB 7.0, which reads partitioned mutation logs.
virtual Future<Optional<RestorableFileSet>> getPartitionedRestoreSet(Version targetVersion) = 0;
// Get an IBackupContainer based on a container spec string
static Reference<IBackupContainer> openContainer(std::string url);
static std::vector<std::string> getURLFormats();

View File

@ -695,6 +695,9 @@ struct TLogVersion {
UNSET = 0,
// Everything between BEGIN and END should be densely packed, so that we
// can iterate over them easily.
// V3 was the introduction of spill by reference;
// V4 changed how data gets written to satellite TLogs so that we can peek from them;
// V5 merged reference and value spilling
// V1 = 1, // 4.6 is dispatched to via 6.0
V2 = 2, // 6.0
V3 = 3, // 6.1
@ -978,13 +981,14 @@ struct WorkerBackupStatus {
LogEpoch epoch;
Version version;
Tag tag;
int32_t totalTags;
WorkerBackupStatus() : epoch(0), version(invalidVersion) {}
WorkerBackupStatus(LogEpoch e, Version v, Tag t) : epoch(e), version(v), tag(t) {}
WorkerBackupStatus(LogEpoch e, Version v, Tag t, int32_t total) : epoch(e), version(v), tag(t), totalTags(total) {}
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, epoch, version, tag);
serializer(ar, epoch, version, tag, totalTags);
}
};

View File

@ -612,7 +612,8 @@ namespace fileBackup {
struct LogFileWriter {
static const std::string &FFs;
LogFileWriter(Reference<IBackupFile> file = Reference<IBackupFile>(), int blockSize = 0) : file(file), blockSize(blockSize), blockEnd(0), fileVersion(2001) {}
LogFileWriter(Reference<IBackupFile> file = Reference<IBackupFile>(), int blockSize = 0)
: file(file), blockSize(blockSize), blockEnd(0) {}
// Start a new block if needed, then write the key and value
ACTOR static Future<Void> writeKV_impl(LogFileWriter *self, Key k, Value v) {
@ -629,8 +630,8 @@ namespace fileBackup {
// Set new blockEnd
self->blockEnd += self->blockSize;
// write Header
wait(self->file->append((uint8_t *)&self->fileVersion, sizeof(self->fileVersion)));
// write the block header
wait(self->file->append((uint8_t *)&BACKUP_AGENT_MLOG_VERSION, sizeof(BACKUP_AGENT_MLOG_VERSION)));
}
wait(self->file->appendStringRefWithLen(k));
@ -650,7 +651,6 @@ namespace fileBackup {
private:
int64_t blockEnd;
uint32_t fileVersion;
};
ACTOR Future<Standalone<VectorRef<KeyValueRef>>> decodeLogFileBlock(Reference<IAsyncFile> file, int64_t offset, int len) {
@ -663,8 +663,8 @@ namespace fileBackup {
state StringRefReader reader(buf, restore_corrupted_data());
try {
// Read header, currently only decoding version 2001
if(reader.consume<int32_t>() != 2001)
// Read header, currently only decoding version BACKUP_AGENT_MLOG_VERSION
if(reader.consume<int32_t>() != BACKUP_AGENT_MLOG_VERSION)
throw restore_unsupported_file_version();
// Read k/v pairs. Block ends either at end of last value exactly or with 0xFF as first key len byte.
@ -2388,7 +2388,8 @@ namespace fileBackup {
// Check if backup worker is enabled
DatabaseConfiguration dbConfig = wait(getDatabaseConfiguration(cx));
if (!dbConfig.backupWorkerEnabled) {
state bool backupWorkerEnabled = dbConfig.backupWorkerEnabled;
if (!backupWorkerEnabled) {
wait(success(changeConfig(cx, "backup_worker_enabled:=1", true)));
}
@ -2404,7 +2405,12 @@ namespace fileBackup {
state Future<Optional<Value>> started = tr->get(backupStartedKey);
state Future<Optional<Value>> taskStarted = tr->get(config.allWorkerStarted().key);
wait(success(started) && success(taskStarted));
state Future<Optional<bool>> partitionedLog = config.partitionedLogEnabled().get(tr);
wait(success(started) && success(taskStarted) && success(partitionedLog));
if (!partitionedLog.get().present() || !partitionedLog.get().get()) {
return Void(); // Skip if not using partitioned logs
}
std::vector<std::pair<UID, Version>> ids;
if (started.get().present()) {
@ -2420,6 +2426,9 @@ namespace fileBackup {
}
tr->set(backupStartedKey, encodeBackupStartedValue(ids));
if (backupWorkerEnabled) {
config.backupWorkerEnabled().set(tr, true);
}
// The task may be restarted. Set the watch if started key has NOT been set.
if (!taskStarted.get().present()) {
@ -3595,7 +3604,10 @@ public:
}
}
ACTOR static Future<Void> submitBackup(FileBackupAgent* backupAgent, Reference<ReadYourWritesTransaction> tr, Key outContainer, int snapshotIntervalSeconds, std::string tagName, Standalone<VectorRef<KeyRangeRef>> backupRanges, bool stopWhenDone) {
ACTOR static Future<Void> submitBackup(FileBackupAgent* backupAgent, Reference<ReadYourWritesTransaction> tr,
Key outContainer, int snapshotIntervalSeconds, std::string tagName,
Standalone<VectorRef<KeyRangeRef>> backupRanges, bool stopWhenDone,
bool partitionedLog) {
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
tr->setOption(FDBTransactionOptions::COMMIT_ON_FIRST_PROXY);
@ -3696,6 +3708,7 @@ public:
config.stopWhenDone().set(tr, stopWhenDone);
config.backupRanges().set(tr, normalizedRanges);
config.snapshotIntervalSeconds().set(tr, snapshotIntervalSeconds);
config.partitionedLogEnabled().set(tr, partitionedLog);
Key taskKey = wait(fileBackup::StartFullBackupTaskFunc::addTask(tr, backupAgent->taskBucket, uid, TaskCompletionKey::noSignal()));
@ -4440,8 +4453,12 @@ Future<ERestoreState> FileBackupAgent::waitRestore(Database cx, Key tagName, boo
return FileBackupAgentImpl::waitRestore(cx, tagName, verbose);
};
Future<Void> FileBackupAgent::submitBackup(Reference<ReadYourWritesTransaction> tr, Key outContainer, int snapshotIntervalSeconds, std::string tagName, Standalone<VectorRef<KeyRangeRef>> backupRanges, bool stopWhenDone) {
return FileBackupAgentImpl::submitBackup(this, tr, outContainer, snapshotIntervalSeconds, tagName, backupRanges, stopWhenDone);
Future<Void> FileBackupAgent::submitBackup(Reference<ReadYourWritesTransaction> tr, Key outContainer,
int snapshotIntervalSeconds, std::string tagName,
Standalone<VectorRef<KeyRangeRef>> backupRanges, bool stopWhenDone,
bool partitionedLog) {
return FileBackupAgentImpl::submitBackup(this, tr, outContainer, snapshotIntervalSeconds, tagName, backupRanges,
stopWhenDone, partitionedLog);
}
Future<Void> FileBackupAgent::discontinueBackup(Reference<ReadYourWritesTransaction> tr, Key tagName){

View File

@ -209,6 +209,8 @@ struct RestoreAsset {
KeyRange range; // Only use mutations in range
int fileIndex;
// Partition ID for mutation log files, which is also encoded in the filename of mutation logs.
int partitionId = -1;
std::string filename;
int64_t offset;
int64_t len;
@ -218,12 +220,12 @@ struct RestoreAsset {
RestoreAsset() = default;
bool operator==(const RestoreAsset& r) const {
return fileIndex == r.fileIndex && filename == r.filename && offset == r.offset && len == r.len &&
beginVersion == r.beginVersion && endVersion == r.endVersion && range == r.range;
return beginVersion == r.beginVersion && endVersion == r.endVersion && range == r.range &&
fileIndex == r.fileIndex && partitionId == r.partitionId && filename == r.filename &&
offset == r.offset && len == r.len;
}
bool operator!=(const RestoreAsset& r) const {
return fileIndex != r.fileIndex || filename != r.filename || offset != r.offset || len != r.len ||
beginVersion != r.beginVersion || endVersion != r.endVersion || range != r.range;
return !(*this == r);
}
bool operator<(const RestoreAsset& r) const {
return std::make_tuple(fileIndex, filename, offset, len, beginVersion, endVersion, range.begin, range.end) <
@ -233,14 +235,14 @@ struct RestoreAsset {
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, beginVersion, endVersion, range, filename, fileIndex, offset, len, uid);
serializer(ar, beginVersion, endVersion, range, filename, fileIndex, partitionId, offset, len, uid);
}
std::string toString() {
std::stringstream ss;
ss << "UID:" << uid.toString() << " begin:" << beginVersion << " end:" << endVersion
<< " range:" << range.toString() << " filename:" << filename << " fileIndex:" << fileIndex
<< " offset:" << offset << " len:" << len;
<< " partitionId:" << partitionId << " offset:" << offset << " len:" << len;
return ss.str();
}
@ -269,6 +271,10 @@ struct LoadingParam {
return (isRangeFile < r.isRangeFile) || (isRangeFile == r.isRangeFile && asset < r.asset);
}
bool isPartitionedLog() const {
return !isRangeFile && asset.partitionId >= 0;
}
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, isRangeFile, url, rangeVersion, blockSize, asset);
@ -447,26 +453,28 @@ struct RestoreSendVersionedMutationsRequest : TimedRequest {
Version prevVersion, version; // version is the commitVersion of the mutation vector.
bool isRangeFile;
MutationsVec mutations; // All mutations at the same version parsed by one loader
SubSequenceVec subs; // Sub-sequence number for mutations
ReplyPromise<RestoreCommonReply> reply;
RestoreSendVersionedMutationsRequest() = default;
explicit RestoreSendVersionedMutationsRequest(int batchIndex, const RestoreAsset& asset, Version prevVersion,
Version version, bool isRangeFile, MutationsVec mutations)
Version version, bool isRangeFile, MutationsVec mutations,
SubSequenceVec subs)
: batchIndex(batchIndex), asset(asset), prevVersion(prevVersion), version(version), isRangeFile(isRangeFile),
mutations(mutations) {}
mutations(mutations), subs(subs) {}
std::string toString() {
std::stringstream ss;
ss << "VersionBatchIndex:" << batchIndex << "RestoreAsset:" << asset.toString()
<< " prevVersion:" << prevVersion << " version:" << version << " isRangeFile:" << isRangeFile
<< " mutations.size:" << mutations.size();
<< " mutations.size:" << mutations.size() << " subs.size:" << subs.size();
return ss.str();
}
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, batchIndex, asset, prevVersion, version, isRangeFile, mutations, reply);
serializer(ar, batchIndex, asset, prevVersion, version, isRangeFile, mutations, subs, reply);
}
};

View File

@ -35,40 +35,74 @@ void BackupProgress::addBackupStatus(const WorkerBackupStatus& status) {
} else {
it.insert(lb, { status.tag, status.version });
}
auto tagIt = epochTags.find(status.epoch);
if (tagIt == epochTags.end()) {
epochTags.insert({ status.epoch, status.totalTags });
} else {
ASSERT(status.totalTags == tagIt->second);
}
}
std::map<std::pair<LogEpoch, Version>, std::map<Tag, Version>> BackupProgress::getUnfinishedBackup() {
std::map<std::pair<LogEpoch, Version>, std::map<Tag, Version>> toRecruit;
void BackupProgress::updateTagVersions(std::map<Tag, Version>* tagVersions, std::set<Tag>* tags,
const std::map<Tag, Version>& progress, Version endVersion, LogEpoch epoch) {
for (const auto& [tag, savedVersion] : progress) {
tags->erase(tag);
if (savedVersion < endVersion - 1) {
tagVersions->insert({ tag, savedVersion + 1 });
TraceEvent("BackupVersionRange", dbgid)
.detail("OldEpoch", epoch)
.detail("Tag", tag.toString())
.detail("BeginVersion", savedVersion + 1)
.detail("EndVersion", endVersion);
}
}
}
std::map<std::tuple<LogEpoch, Version, int>, std::map<Tag, Version>> BackupProgress::getUnfinishedBackup() {
std::map<std::tuple<LogEpoch, Version, int>, std::map<Tag, Version>> toRecruit;
if (!backupStartedValue.present()) return toRecruit; // No active backups
for (const auto& [epoch, info] : epochInfos) {
std::set<Tag> tags = enumerateLogRouterTags(info.logRouterTags);
std::map<Tag, Version> tagVersions;
auto progressIt = progress.find(epoch);
if (progressIt != progress.end()) {
for (const auto& [tag, savedVersion] : progressIt->second) {
tags.erase(tag);
if (savedVersion < info.epochEnd - 1) {
tagVersions.insert({ tag, savedVersion + 1 });
TraceEvent("BW", dbgid)
.detail("OldEpoch", epoch)
.detail("Tag", tag.toString())
.detail("BeginVersion", savedVersion + 1)
.detail("EndVersion", info.epochEnd);
auto progressIt = progress.lower_bound(epoch);
if (progressIt != progress.end() && progressIt->first == epoch) {
updateTagVersions(&tagVersions, &tags, progressIt->second, info.epochEnd, epoch);
} else {
auto rit = std::find_if(
progress.rbegin(), progress.rend(),
[epoch = epoch](const std::pair<LogEpoch, std::map<Tag, Version>>& p) { return p.first < epoch; });
if (!(rit == progress.rend())) {
// A partial recovery can result in empty epoch that copies previous
// epoch's version range. In this case, we should check previous
// epoch's savedVersion.
int savedMore = 0;
for (auto [tag, version] : rit->second) {
if (version >= info.epochBegin) {
savedMore++;
}
}
if (savedMore > 0) {
// The logRouterTags are the same
// ASSERT(info.logRouterTags == epochTags[rit->first]);
updateTagVersions(&tagVersions, &tags, rit->second, info.epochEnd, epoch);
}
}
}
for (const Tag tag : tags) { // tags without progress data
tagVersions.insert({ tag, info.epochBegin });
TraceEvent("BW", dbgid)
TraceEvent("BackupVersionRange", dbgid)
.detail("OldEpoch", epoch)
.detail("Tag", tag.toString())
.detail("BeginVersion", info.epochBegin)
.detail("EndVersion", info.epochEnd);
}
if (!tagVersions.empty()) {
toRecruit[{ epoch, info.epochEnd }] = tagVersions;
toRecruit[{ epoch, info.epochEnd, info.logRouterTags }] = tagVersions;
}
}
return toRecruit;
@ -94,10 +128,11 @@ ACTOR Future<Void> getBackupProgress(Database cx, UID dbgid, Reference<BackupPro
const WorkerBackupStatus status = decodeBackupProgressValue(it.value);
bStatus->addBackupStatus(status);
TraceEvent("GotBackupProgress", dbgid)
.detail("W", workerID)
.detail("BackupWorker", workerID)
.detail("Epoch", status.epoch)
.detail("Version", status.version)
.detail("Tag", status.tag.toString());
.detail("Tag", status.tag.toString())
.detail("TotalTags", status.totalTags);
}
return Void();
} catch (Error& e) {
@ -115,21 +150,23 @@ TEST_CASE("/BackupProgress/Unfinished") {
BackupProgress progress(UID(0, 0), epochInfos);
progress.setBackupStartedValue(Optional<Value>(LiteralStringRef("1")));
std::map<std::pair<LogEpoch, Version>, std::map<Tag, Version>> unfinished = progress.getUnfinishedBackup();
std::map<std::tuple<LogEpoch, Version, int>, std::map<Tag, Version>> unfinished = progress.getUnfinishedBackup();
ASSERT(unfinished.size() == 1);
for (const auto [epochVersion, tagVersion] : unfinished) {
ASSERT(epochVersion.first == epoch1 && epochVersion.second == end1);
for (const auto [epochVersionCount, tagVersion] : unfinished) {
ASSERT(std::get<0>(epochVersionCount) == epoch1 && std::get<1>(epochVersionCount) == end1 &&
std::get<2>(epochVersionCount) == 1);
ASSERT(tagVersion.size() == 1 && tagVersion.begin()->first == tag1 && tagVersion.begin()->second == begin1);
}
const int saved1 = 50;
WorkerBackupStatus status1(epoch1, saved1, tag1);
const int saved1 = 50, totalTags = 1;
WorkerBackupStatus status1(epoch1, saved1, tag1, totalTags);
progress.addBackupStatus(status1);
unfinished = progress.getUnfinishedBackup();
ASSERT(unfinished.size() == 1);
for (const auto [epochVersion, tagVersion] : unfinished) {
ASSERT(epochVersion.first == epoch1 && epochVersion.second == end1);
for (const auto [epochVersionCount, tagVersion] : unfinished) {
ASSERT(std::get<0>(epochVersionCount) == epoch1 && std::get<1>(epochVersionCount) == end1 &&
std::get<2>(epochVersionCount) == 1);
ASSERT(tagVersion.size() == 1 && tagVersion.begin()->first == tag1 && tagVersion.begin()->second == saved1 + 1);
}

View File

@ -25,6 +25,8 @@
#define FDBSERVER_BACKUPPROGRESS_ACTOR_H
#include <map>
#include <tuple>
#include "fdbclient/FDBTypes.h"
#include "fdbserver/LogSystem.h"
#include "flow/Arena.h"
@ -41,7 +43,7 @@ public:
// savedVersion is used.
void addBackupStatus(const WorkerBackupStatus& status);
// Returns a map of pair<Epoch, endVersion> : map<tag, savedVersion>, so that
// Returns a map of tuple<Epoch, endVersion, logRouterTags> : map<tag, savedVersion>, so that
// the backup range should be [savedVersion + 1, endVersion) for the "tag" of the "Epoch".
//
// Specifically, the backup ranges for each old epoch are:
@ -49,7 +51,7 @@ public:
// backup [epochBegin, endVersion)
// else if savedVersion < endVersion - 1 = knownCommittedVersion
// backup [savedVersion + 1, endVersion)
std::map<std::pair<LogEpoch, Version>, std::map<Tag, Version>> getUnfinishedBackup();
std::map<std::tuple<LogEpoch, Version, int>, std::map<Tag, Version>> getUnfinishedBackup();
// Set the value for "backupStartedKey"
void setBackupStartedValue(Optional<Value> value) {
@ -76,6 +78,11 @@ private:
return tags;
}
// For each tag in progress, the saved version is smaller than endVersion - 1,
// add {tag, savedVersion+1} to tagVersions and remove the tag from "tags".
void updateTagVersions(std::map<Tag, Version>* tagVersions, std::set<Tag>* tags,
const std::map<Tag, Version>& progress, Version endVersion, LogEpoch epoch);
const UID dbgid;
// Note this MUST be iterated in ascending order.
@ -86,6 +93,10 @@ private:
// the gap. "progress" MUST be iterated in ascending order.
std::map<LogEpoch, std::map<Tag, Version>> progress;
// LogRouterTags for each epoch obtained by decoding backup progress from
// the system keyspace.
std::map<LogEpoch, int32_t> epochTags;
// Value of the "backupStartedKey".
Optional<Value> backupStartedValue;
};

View File

@ -67,23 +67,140 @@ struct VersionedMessage {
struct BackupData {
const UID myId;
const Tag tag; // LogRouter tag for this worker, i.e., (-2, i)
const int totalTags; // Total log router tags
const Version startVersion;
const Optional<Version> endVersion; // old epoch's end version (inclusive), or empty for current epoch
const LogEpoch recruitedEpoch;
const LogEpoch backupEpoch;
const LogEpoch recruitedEpoch; // current epoch whose tLogs are receiving mutations
const LogEpoch backupEpoch; // the epoch workers should pull mutations
LogEpoch oldestBackupEpoch = 0; // oldest epoch that still has data on tLogs for backup to pull
Version minKnownCommittedVersion;
Version savedVersion;
AsyncVar<Reference<ILogSystem>> logSystem;
Database cx;
std::vector<VersionedMessage> messages;
AsyncVar<bool> pullFinished;
NotifiedVersion pulledVersion;
bool pulling = false;
bool stopped = false;
bool exitEarly = false; // If the worker is on an old epoch and all backups starts a version >= the endVersion
struct PerBackupInfo {
PerBackupInfo() = default;
PerBackupInfo(BackupData* data, Version v) : self(data), startVersion(v) {}
PerBackupInfo(BackupData* data, UID uid, Version v) : self(data), startVersion(v) {
// Open the container and get key ranges
BackupConfig config(uid);
container = config.backupContainer().get(data->cx);
ranges = config.backupRanges().get(data->cx);
if (self->backupEpoch == self->recruitedEpoch) {
// Only current epoch's worker update the number of backup workers.
updateWorker = _updateStartedWorkers(this, data, uid);
}
TraceEvent("BackupWorkerAddJob", data->myId).detail("BackupID", uid).detail("Version", v);
}
bool isRunning() {
return container.isReady() && ranges.isReady() && !stopped;
void stop() {
stopped = true;
updateWorker = Void(); // cancel actors
}
void cancelUpdater() { updateWorker = Void(); }
bool isReady() const {
return stopped || (container.isReady() && ranges.isReady());
}
Future<Void> waitReady() {
if (stopped) return Void();
return _waitReady(this);
}
ACTOR static Future<Void> _waitReady(PerBackupInfo* info) {
wait(success(info->container) && success(info->ranges));
return Void();
}
// Update the number of backup workers in the BackupConfig. Each worker
// writes (epoch, tag.id) into the key. Worker 0 monitors the key and once
// all workers have updated the key, this backup is considered as started
// (i.e., the "submitBackup" call is successful). Worker 0 then sets
// the "allWorkerStarted" flag, which in turn unblocks
// StartFullBackupTaskFunc::_execute.
ACTOR static Future<Void> _updateStartedWorkers(PerBackupInfo* info, BackupData* self, UID uid) {
state BackupConfig config(uid);
state Future<Void> watchFuture;
state bool updated = false;
state bool firstWorker = info->self->tag.id == 0;
state bool allUpdated = false;
state Optional<std::vector<std::pair<int64_t, int64_t>>> workers;
state Reference<ReadYourWritesTransaction> tr(new ReadYourWritesTransaction(self->cx));
loop {
try {
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE);
Optional<std::vector<std::pair<int64_t, int64_t>>> tmp =
wait(config.startedBackupWorkers().get(tr));
workers = tmp;
if (!updated) {
if (workers.present()) {
workers.get().emplace_back(self->recruitedEpoch, (int64_t)self->tag.id);
} else {
std::vector<std::pair<int64_t, int64_t>> v(1, { self->recruitedEpoch, self->tag.id });
workers = Optional<std::vector<std::pair<int64_t, int64_t>>>(v);
}
}
if (firstWorker) {
if (!workers.present()) {
TraceEvent("BackupWorkerDetectAbortedJob", self->myId).detail("BackupID", uid);
return Void();
}
ASSERT(workers.present() && workers.get().size() > 0);
std::vector<std::pair<int64_t, int64_t>>& v = workers.get();
v.erase(std::remove_if(v.begin(), v.end(),
[epoch = self->recruitedEpoch](const std::pair<int64_t, int64_t>& p) {
return p.first != epoch;
}),
v.end());
std::set<int64_t> tags;
for (auto p : v) {
tags.insert(p.second);
}
if (self->totalTags == tags.size()) {
config.allWorkerStarted().set(tr, true);
allUpdated = true;
} else {
// monitor all workers' updates
watchFuture = tr->watch(config.startedBackupWorkers().key);
}
ASSERT(workers.present() && workers.get().size() > 0);
if (!updated) {
config.startedBackupWorkers().set(tr, workers.get());
}
for (auto p : workers.get()) {
TraceEvent("BackupWorkerDebug", self->myId).detail("Epoch", p.first).detail("TagID", p.second);
}
wait(tr->commit());
updated = true; // Only set to true after commit.
if (allUpdated) {
break;
}
wait(watchFuture);
tr->reset();
} else {
ASSERT(workers.present() && workers.get().size() > 0);
config.startedBackupWorkers().set(tr, workers.get());
wait(tr->commit());
break;
}
} catch (Error& e) {
wait(tr->onError(e));
allUpdated = false;
}
}
TraceEvent("BackupWorkerSetReady", self->myId).detail("BackupID", uid).detail("TagId", self->tag.id);
return Void();
}
BackupData* self = nullptr;
@ -91,22 +208,23 @@ struct BackupData {
Version lastSavedVersion = invalidVersion;
Future<Optional<Reference<IBackupContainer>>> container;
Future<Optional<std::vector<KeyRange>>> ranges; // Key ranges of this backup
bool allWorkerStarted = false; // Only worker with Tag(-2,0) uses & sets this field
Future<Void> updateWorker;
bool stopped = false; // Is the backup stopped?
};
std::map<UID, PerBackupInfo> backups; // Backup UID to infos
AsyncTrigger changedTrigger;
AsyncTrigger doneTrigger;
CounterCollection cc;
Future<Void> logger;
explicit BackupData(UID id, Reference<AsyncVar<ServerDBInfo>> db, const InitializeBackupRequest& req)
: myId(id), tag(req.routerTag), startVersion(req.startVersion), endVersion(req.endVersion),
recruitedEpoch(req.recruitedEpoch), backupEpoch(req.backupEpoch), minKnownCommittedVersion(invalidVersion),
savedVersion(invalidVersion), cc("BackupWorker", myId.toString()) {
: myId(id), tag(req.routerTag), totalTags(req.totalTags), startVersion(req.startVersion),
endVersion(req.endVersion), recruitedEpoch(req.recruitedEpoch), backupEpoch(req.backupEpoch),
minKnownCommittedVersion(invalidVersion), savedVersion(req.startVersion - 1),
cc("BackupWorker", myId.toString()), pulledVersion(0) {
cx = openDBOnServer(db, TaskPriority::DefaultEndpoint, true, true);
pullFinished.set(false);
specialCounter(cc, "SavedVersion", [this]() { return this->savedVersion; });
specialCounter(cc, "MinKnownCommittedVersion", [this]() { return this->minKnownCommittedVersion; });
@ -115,6 +233,18 @@ struct BackupData {
"BackupWorkerMetrics");
}
bool pullFinished() const {
return endVersion.present() && pulledVersion.get() > endVersion.get();
}
bool allMessageSaved() const {
return (endVersion.present() && savedVersion >= endVersion.get()) || stopped || exitEarly;
}
Version maxPopVersion() const {
return endVersion.present() ? endVersion.get() : minKnownCommittedVersion;
}
// Inserts a backup's single range into rangeMap.
template <class T>
void insertRange(KeyRangeMap<std::set<T>>& keyRangeMap, KeyRangeRef range, T value) {
@ -143,22 +273,35 @@ struct BackupData {
}
void pop() {
const LogEpoch oldest = logSystem.get()->getOldestBackupEpoch();
if (backupEpoch > oldest) {
if (backupEpoch > oldestBackupEpoch || stopped) {
// Defer pop if old epoch hasn't finished popping yet.
// If stopped because of displacement, do NOT pop as the progress may
// not be saved in a timely fashion. As a result, next epoch may still
// need to read mutations in the version range. Let the next epoch's
// worker do the pop instead.
TraceEvent("BackupWorkerPopDeferred", myId)
.suppressFor(1.0)
.detail("BackupEpoch", backupEpoch)
.detail("OldestEpoch", oldest)
.detail("OldestEpoch", oldestBackupEpoch)
.detail("Version", savedVersion);
return;
}
// ASSERT will be fixed in PR#2642
// ASSERT_WE_THINK(backupEpoch == oldest);
ASSERT_WE_THINK(backupEpoch == oldestBackupEpoch);
const Tag popTag = logSystem.get()->getPseudoPopTag(tag, ProcessClass::BackupClass);
logSystem.get()->pop(savedVersion, popTag);
}
void stop() {
stopped = true;
for (auto& [uid, info] : backups) {
// Cancel the actor. Because container is valid, CANNOT set the
// "stop" flag that will block writing mutation files in
// saveMutationsToFile().
info.cancelUpdater();
}
doneTrigger.trigger();
}
void eraseMessagesAfterEndVersion() {
ASSERT(endVersion.present());
const Version ver = endVersion.get();
@ -180,18 +323,11 @@ struct BackupData {
}
bool modified = false;
for (const auto uidVersion : uidVersions) {
const UID uid = uidVersion.first;
for (const auto [uid, version] : uidVersions) {
auto it = backups.find(uid);
if (it == backups.end()) {
modified = true;
auto inserted = backups.emplace(uid, BackupData::PerBackupInfo(this, uidVersion.second));
// Open the container and get key ranges
BackupConfig config(uid);
inserted.first->second.container = config.backupContainer().get(cx);
inserted.first->second.ranges = config.backupRanges().get(cx);
backups.emplace(uid, BackupData::PerBackupInfo(this, uid, version));
} else {
stopList.erase(uid);
}
@ -200,16 +336,61 @@ struct BackupData {
for (UID uid : stopList) {
auto it = backups.find(uid);
ASSERT(it != backups.end());
it->second.stopped = true;
it->second.stop();
modified = true;
}
if (modified) changedTrigger.trigger();
}
ACTOR static Future<Void> _waitAllInfoReady(BackupData* self) {
std::vector<Future<Void>> all;
for (auto it = self->backups.begin(); it != self->backups.end(); ) {
if (it->second.stopped) {
TraceEvent("BackupWorkerRemoveStoppedContainer", self->myId).detail("BackupId", it->first);
it = self->backups.erase(it);
continue;
}
all.push_back(it->second.waitReady());
it++;
}
wait(waitForAll(all));
return Void();
}
Future<Void> waitAllInfoReady() {
return _waitAllInfoReady(this);
}
bool isAllInfoReady() const {
for (const auto& [uid, info] : backups) {
if (!info.isReady()) return false;
}
return true;
}
ACTOR static Future<Version> _getMinKnownCommittedVersion(BackupData* self) {
loop {
GetReadVersionRequest request(1, GetReadVersionRequest::PRIORITY_DEFAULT |
GetReadVersionRequest::FLAG_USE_MIN_KNOWN_COMMITTED_VERSION);
choose {
when(wait(self->cx->onMasterProxiesChanged())) {}
when(GetReadVersionReply reply = wait(loadBalance(self->cx->getMasterProxies(false),
&MasterProxyInterface::getConsistentReadVersion,
request, self->cx->taskID))) {
return reply.version;
}
}
}
}
Future<Version> getMinKnownCommittedVersion() { return _getMinKnownCommittedVersion(this); }
};
// Monitors "backupStartedKey". If "started" is true, wait until the key is set;
// otherwise, wait until the key is cleared.
ACTOR Future<Void> monitorBackupStartedKeyChanges(BackupData* self, bool started) {
// otherwise, wait until the key is cleared. If "watch" is false, do not perform
// the wait for key set/clear events. Returns if key present.
ACTOR Future<bool> monitorBackupStartedKeyChanges(BackupData* self, bool started, bool watch) {
loop {
state ReadYourWritesTransaction tr(self->cx);
@ -219,23 +400,29 @@ ACTOR Future<Void> monitorBackupStartedKeyChanges(BackupData* self, bool started
tr.setOption(FDBTransactionOptions::LOCK_AWARE);
Optional<Value> value = wait(tr.get(backupStartedKey));
std::vector<std::pair<UID, Version>> uidVersions;
bool shouldExit = self->endVersion.present();
if (value.present()) {
uidVersions = decodeBackupStartedValue(value.get());
TraceEvent e("BackupWorkerGotStartKey", self->myId);
int i = 1;
for (auto uidVersion : uidVersions) {
e.detail(format("BackupID%d", i), uidVersion.first)
.detail(format("Version%d", i), uidVersion.second);
for (auto [uid, version] : uidVersions) {
e.detail(format("BackupID%d", i), uid)
.detail(format("Version%d", i), version);
i++;
if (shouldExit && version < self->endVersion.get()) {
shouldExit = false;
}
}
self->exitEarly = shouldExit;
self->onBackupChanges(uidVersions);
if (started) return Void();
if (started || !watch) return true;
} else {
TraceEvent("BackupWorkerEmptyStartKey", self->myId);
self->onBackupChanges(uidVersions);
if (!started) {
return Void();
self->exitEarly = shouldExit;
if (!started || !watch) {
return false;
}
}
@ -250,69 +437,94 @@ ACTOR Future<Void> monitorBackupStartedKeyChanges(BackupData* self, bool started
}
}
// Monitor all backup worker in the recruited epoch has been started. If so,
// set the "allWorkerStarted" key of the BackupConfig to true, which in turn
// unblocks StartFullBackupTaskFunc::_execute. Note only worker with Tag (-2,0)
// runs this actor so that the key is set by one process.
ACTOR Future<Void> monitorAllWorkerStarted(BackupData* self) {
// Set "latestBackupWorkerSavedVersion" key for backups
ACTOR Future<Void> setBackupKeys(BackupData* self, std::map<UID, Version> savedLogVersions) {
state Reference<ReadYourWritesTransaction> tr(new ReadYourWritesTransaction(self->cx));
loop {
wait(delay(SERVER_KNOBS->WORKER_LOGGING_INTERVAL / 2.0) || self->changedTrigger.onTrigger());
if (self->backups.empty()) {
continue;
try {
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
tr->setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE);
state std::vector<Future<Optional<Version>>> prevVersions;
state std::vector<BackupConfig> versionConfigs;
state std::vector<Future<Optional<bool>>> allWorkersReady;
for (const auto [uid, version] : savedLogVersions) {
versionConfigs.emplace_back(uid);
prevVersions.push_back(versionConfigs.back().latestBackupWorkerSavedVersion().get(tr));
allWorkersReady.push_back(versionConfigs.back().allWorkerStarted().get(tr));
}
wait(waitForAll(prevVersions) && waitForAll(allWorkersReady));
for (int i = 0; i < prevVersions.size(); i++) {
if (!allWorkersReady[i].get().present() || !allWorkersReady[i].get().get()) continue;
const Version current = savedLogVersions[versionConfigs[i].getUid()];
if (prevVersions[i].get().present()) {
const Version prev = prevVersions[i].get().get();
if (prev > current) {
TraceEvent(SevWarn, "BackupWorkerVersionInverse", self->myId)
.detail("Prev", prev)
.detail("Current", current);
}
}
if (self->backupEpoch == self->oldestBackupEpoch &&
(!prevVersions[i].get().present() || prevVersions[i].get().get() < current)) {
TraceEvent("BackupWorkerSetVersion", self->myId)
.detail("BackupID", versionConfigs[i].getUid())
.detail("Version", current);
versionConfigs[i].latestBackupWorkerSavedVersion().set(tr, current);
}
}
wait(tr->commit());
return Void();
} catch (Error& e) {
wait(tr->onError(e));
}
}
}
// Note only worker with Tag (-2,0) runs this actor so that the latest saved
// version key is set by one process, which is stored in each BackupConfig in
// the system space. The client can know if a backup is restorable by checking
// log saved version > snapshot version.
ACTOR Future<Void> monitorBackupProgress(BackupData* self) {
state Future<Void> interval;
loop {
interval = delay(SERVER_KNOBS->WORKER_LOGGING_INTERVAL / 2.0);
while (self->backups.empty() || !self->logSystem.get()) {
wait(self->changedTrigger.onTrigger() || self->logSystem.onChange());
}
// check all workers have started by checking their progress is larger
// than the backup's start version.
state Reference<BackupProgress> progress(new BackupProgress(self->myId, {}));
wait(getBackupProgress(self->cx, self->myId, progress));
std::map<Tag, Version> tagVersions = progress->getEpochStatus(self->recruitedEpoch);
state std::map<Tag, Version> tagVersions = progress->getEpochStatus(self->recruitedEpoch);
state std::map<UID, Version> savedLogVersions;
if (tagVersions.size() != self->totalTags) {
wait(interval);
continue;
}
state std::vector<UID> ready;
if (tagVersions.size() == self->logSystem.get()->getLogRouterTags()) {
// Check every version is larger than backup's startVersion
for (auto& uidInfo : self->backups) {
if (uidInfo.second.allWorkerStarted) continue;
bool saved = true;
for (const std::pair<Tag, Version> tv : tagVersions) {
if (tv.second < uidInfo.second.startVersion) {
saved = false;
break;
}
}
if (saved) {
ready.push_back(uidInfo.first);
uidInfo.second.allWorkerStarted = true;
}
}
if (ready.empty()) continue;
// Set "allWorkerStarted" key for ready backups
loop {
state Reference<ReadYourWritesTransaction> tr(new ReadYourWritesTransaction(self->cx));
try {
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
state std::vector<Future<Optional<Value>>> readyValues;
state std::vector<BackupConfig> configs;
for (UID uid : ready) {
configs.emplace_back(uid);
readyValues.push_back(tr->get(configs.back().allWorkerStarted().key));
}
wait(waitForAll(readyValues));
for (int i = 0; i < readyValues.size(); i++) {
if (!readyValues[i].get().present()) {
configs[i].allWorkerStarted().set(tr, true);
TraceEvent("BackupWorkerSetReady", self->myId).detail("BackupID", ready[i].toString());
}
}
wait(tr->commit());
break;
} catch (Error& e) {
wait(tr->onError(e));
// Check every version is larger than backup's startVersion
for (auto& [uid, info] : self->backups) {
if (self->recruitedEpoch == self->oldestBackupEpoch) {
// update update progress so far if previous epochs are done
Version v = std::numeric_limits<Version>::max();
for (const auto [tag, version] : tagVersions) {
v = std::min(v, version);
}
savedLogVersions.emplace(uid, v);
TraceEvent("BackupWorkerSavedBackupVersion", self->myId).detail("BackupID", uid).detail("Version", v);
}
}
Future<Void> setKeys = savedLogVersions.empty() ? Void() : setBackupKeys(self, savedLogVersions);
wait(interval && setKeys);
}
}
@ -322,11 +534,13 @@ ACTOR Future<Void> saveProgress(BackupData* self, Version backupVersion) {
loop {
try {
// It's critical to save progress immediately so that after a master
// recovery, the new master can know the progress so far.
tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE);
tr.setOption(FDBTransactionOptions::LOCK_AWARE);
WorkerBackupStatus status(self->backupEpoch, backupVersion, self->tag);
WorkerBackupStatus status(self->backupEpoch, backupVersion, self->tag, self->totalTags);
tr.set(key, backupProgressValue(status));
tr.addReadConflictRange(singleKeyRange(key));
wait(tr.commit());
@ -373,7 +587,8 @@ ACTOR Future<Void> addMutation(Reference<IBackupFile> logFile, VersionedMessage
}
*blockEnd += blockSize;
// TODO: add block header
// write block Header
wait(logFile->append((uint8_t*)&PARTITIONED_MLOG_VERSION, sizeof(PARTITIONED_MLOG_VERSION)));
}
wait(logFile->append((void*)header.begin(), header.size()));
@ -389,51 +604,46 @@ ACTOR Future<Void> saveMutationsToFile(BackupData* self, Version popVersion, int
state std::vector<Future<Reference<IBackupFile>>> logFileFutures;
state std::vector<Reference<IBackupFile>> logFiles;
state std::vector<int64_t> blockEnds;
state std::set<UID> activeUids; // active Backups' UIDs
state std::vector<UID> activeUids; // active Backups' UIDs
state KeyRangeMap<std::set<int>> keyRangeMap; // range to index in logFileFutures, logFiles, & blockEnds
state std::vector<Standalone<StringRef>> mutations;
state int idx;
// Make sure all backups are ready, otherwise mutations will be lost.
while (!self->isAllInfoReady()) {
wait(self->waitAllInfoReady());
}
for (auto it = self->backups.begin(); it != self->backups.end();) {
if (!it->second.isRunning()) {
if (it->second.stopped) {
TraceEvent("BackupWorkerRemoveStoppedContainer", self->myId).detail("BackupId", it->first);
it = self->backups.erase(it);
} else {
it++;
}
continue;
}
if (!it->second.container.get().present()) {
if (it->second.stopped || !it->second.container.get().present()) {
TraceEvent("BackupWorkerNoContainer", self->myId).detail("BackupId", it->first);
it = self->backups.erase(it);
continue;
}
const int index = logFileFutures.size();
activeUids.insert(it->first);
activeUids.push_back(it->first);
self->insertRanges(keyRangeMap, it->second.ranges.get(), index);
if (it->second.lastSavedVersion == invalidVersion) {
it->second.lastSavedVersion = self->messages[0].getVersion();
it->second.lastSavedVersion =
self->savedVersion > self->startVersion ? self->savedVersion : self->startVersion;
}
logFileFutures.push_back(it->second.container.get().get()->writeTaggedLogFile(
it->second.lastSavedVersion, popVersion + 1, blockSize, self->tag.id));
it->second.lastSavedVersion, popVersion + 1, blockSize, self->tag.id, self->totalTags));
it++;
}
if (activeUids.empty()) {
// stop early if there is no active backups
TraceEvent("BackupWorkerSkip", self->myId).detail("Count", numMsg);
return Void();
}
keyRangeMap.coalesce(allKeys);
wait(waitForAll(logFileFutures));
std::transform(logFileFutures.begin(), logFileFutures.end(), std::back_inserter(logFiles),
[](const Future<Reference<IBackupFile>>& f) { return f.get(); });
for (const auto& file : logFiles) {
ASSERT(activeUids.size() == logFiles.size());
for (int i = 0; i < logFiles.size(); i++) {
TraceEvent("OpenMutationFile", self->myId)
.detail("BackupID", activeUids[i])
.detail("TagId", self->tag.id)
.detail("File", file->getFileName());
.detail("File", logFiles[i]->getFileName());
}
blockEnds = std::vector<int64_t>(logFiles.size(), 0);
@ -442,6 +652,14 @@ ACTOR Future<Void> saveMutationsToFile(BackupData* self, Version popVersion, int
MutationRef m;
if (!message.isBackupMessage(&m)) continue;
if (debugMutation("addMutation", message.version.version, m)) {
TraceEvent("BackupWorkerDebug", self->myId)
.detail("Version", message.version.toString())
.detail("Mutation", m.toString())
.detail("KCV", self->minKnownCommittedVersion)
.detail("SavedVersion", self->savedVersion);
}
std::vector<Future<Void>> adds;
if (m.type != MutationRef::Type::ClearRange) {
for (int index : keyRangeMap[m.param1]) {
@ -493,37 +711,38 @@ ACTOR Future<Void> uploadData(BackupData* self) {
state Version popVersion = invalidVersion;
loop {
if (self->endVersion.present() && self->savedVersion >= self->endVersion.get()) {
self->messages.clear();
return Void();
}
// Too large uploadDelay will delay popping tLog data for too long.
state Future<Void> uploadDelay = delay(SERVER_KNOBS->BACKUP_UPLOAD_DELAY);
const Version maxPopVersion =
self->endVersion.present() ? self->endVersion.get() : self->minKnownCommittedVersion;
state int numMsg = 0;
Version lastPopVersion = popVersion;
if (self->messages.empty()) {
// Even though messages is empty, we still want to advance popVersion.
popVersion = std::max(popVersion, maxPopVersion);
if (!self->endVersion.present()) {
popVersion = std::max(popVersion, self->minKnownCommittedVersion);
}
} else {
state int numMsg = 0;
for (const auto& message : self->messages) {
// message may be prefetched in peek; uncommitted message should not be uploaded.
if (message.getVersion() > maxPopVersion) break;
if (message.getVersion() > self->maxPopVersion()) break;
popVersion = std::max(popVersion, message.getVersion());
numMsg++;
}
if (numMsg > 0) {
wait(saveMutationsToFile(self, popVersion, numMsg));
self->messages.erase(self->messages.begin(), self->messages.begin() + numMsg);
}
}
if (self->pullFinished.get() && self->messages.empty()) {
// Advance popVersion to the endVersion to avoid gap between last
// message version and the endVersion.
if (self->pullFinished()) {
popVersion = self->endVersion.get();
}
if (((numMsg > 0 || popVersion > lastPopVersion) && self->pulling) || self->pullFinished()) {
TraceEvent("BackupWorkerSave", self->myId)
.detail("Version", popVersion)
.detail("MsgQ", self->messages.size());
// save an empty file for old epochs so that log file versions are continuous
wait(saveMutationsToFile(self, popVersion, numMsg));
self->messages.erase(self->messages.begin(), self->messages.begin() + numMsg);
}
// If transition into NOOP mode, should clear messages
if (!self->pulling) self->messages.clear();
if (popVersion > self->savedVersion) {
wait(saveProgress(self, popVersion));
@ -535,8 +754,13 @@ ACTOR Future<Void> uploadData(BackupData* self) {
self->pop();
}
if (!self->pullFinished.get()) {
wait(uploadDelay || self->pullFinished.onChange());
if (self->allMessageSaved()) {
self->messages.clear();
return Void();
}
if (!self->pullFinished()) {
wait(uploadDelay || self->doneTrigger.onTrigger());
}
}
}
@ -545,7 +769,7 @@ ACTOR Future<Void> uploadData(BackupData* self) {
ACTOR Future<Void> pullAsyncData(BackupData* self) {
state Future<Void> logSystemChange = Void();
state Reference<ILogSystem::IPeekCursor> r;
state Version tagAt = std::max(self->startVersion, self->savedVersion);
state Version tagAt = std::max(self->pulledVersion.get(), std::max(self->startVersion, self->savedVersion));
TraceEvent("BackupWorkerPull", self->myId);
loop {
@ -572,51 +796,64 @@ ACTOR Future<Void> pullAsyncData(BackupData* self) {
}
tagAt = r->version().version;
self->pulledVersion.set(tagAt);
TraceEvent("BackupWorkerGot", self->myId).suppressFor(1.0).detail("V", tagAt);
if (self->endVersion.present() && tagAt > self->endVersion.get()) {
if (self->pullFinished()) {
self->eraseMessagesAfterEndVersion();
self->doneTrigger.trigger();
TraceEvent("BackupWorkerFinishPull", self->myId)
.detail("Tag", self->tag.toString())
.detail("VersionGot", tagAt)
.detail("EndVersion", self->endVersion.get())
.detail("MsgQ", self->messages.size());
self->pullFinished.set(true);
return Void();
}
wait(yield());
}
}
ACTOR Future<Void> monitorBackupKeyOrPullData(BackupData* self) {
state Future<Void> started, pullFinished;
ACTOR Future<Void> monitorBackupKeyOrPullData(BackupData* self, bool keyPresent) {
state Future<Void> pullFinished = Void();
loop {
started = monitorBackupStartedKeyChanges(self, true);
loop {
GetReadVersionRequest request(1, GetReadVersionRequest::PRIORITY_DEFAULT |
GetReadVersionRequest::FLAG_USE_MIN_KNOWN_COMMITTED_VERSION);
state Future<bool> present = monitorBackupStartedKeyChanges(self, !keyPresent, /*watch=*/true);
if (keyPresent) {
pullFinished = pullAsyncData(self);
self->pulling = true;
wait(success(present) || pullFinished);
if (pullFinished.isReady()) {
self->pulling = false;
return Void(); // backup is done for some old epoch.
}
choose {
when(wait(started)) { break; }
when(wait(self->cx->onMasterProxiesChanged())) {}
when(GetReadVersionReply reply = wait(loadBalance(self->cx->getMasterProxies(false),
&MasterProxyInterface::getConsistentReadVersion,
request, self->cx->taskID))) {
self->savedVersion = std::max(reply.version, self->savedVersion);
self->minKnownCommittedVersion = std::max(reply.version, self->minKnownCommittedVersion);
TraceEvent("BackupWorkerNoopPop", self->myId).detail("SavedVersion", self->savedVersion);
self->pop(); // Pop while the worker is in this NOOP state.
wait(delay(SERVER_KNOBS->BACKUP_NOOP_POP_DELAY, self->cx->taskID));
// Even though the snapshot is done, mutation logs may not be written
// out yet. We need to make sure mutations up to this point is written.
Version currentVersion = wait(self->getMinKnownCommittedVersion());
wait(self->pulledVersion.whenAtLeast(currentVersion));
pullFinished = Future<Void>(); // cancels pullAsyncData()
self->pulling = false;
TraceEvent("BackupWorkerPaused", self->myId);
} else {
// Backup key is not present, enter this NOOP POP mode.
state Future<Version> committedVersion = self->getMinKnownCommittedVersion();
loop choose {
when(wait(success(present))) { break; }
when(wait(success(committedVersion) || delay(SERVER_KNOBS->BACKUP_NOOP_POP_DELAY, self->cx->taskID))) {
if (committedVersion.isReady()) {
self->savedVersion = std::max(committedVersion.get(), self->savedVersion);
self->minKnownCommittedVersion =
std::max(committedVersion.get(), self->minKnownCommittedVersion);
TraceEvent("BackupWorkerNoopPop", self->myId).detail("SavedVersion", self->savedVersion);
self->pop(); // Pop while the worker is in this NOOP state.
committedVersion = Never();
} else {
committedVersion = self->getMinKnownCommittedVersion();
}
}
}
}
Future<Void> stopped = monitorBackupStartedKeyChanges(self, false);
pullFinished = pullAsyncData(self);
wait(stopped || pullFinished);
if (pullFinished.isReady()) return Void(); // backup is done for some old epoch.
pullFinished = Future<Void>(); // cancels pullAsyncData()
TraceEvent("BackupWorkerPaused", self->myId);
keyPresent = !keyPresent;
}
}
@ -644,37 +881,44 @@ ACTOR Future<Void> backupWorker(BackupInterface interf, InitializeBackupRequest
state PromiseStream<Future<Void>> addActor;
state Future<Void> error = actorCollection(addActor.getFuture());
state Future<Void> dbInfoChange = Void();
state Future<Void> pull;
state Future<Void> done;
TraceEvent("BackupWorkerStart", self.myId)
.detail("Tag", req.routerTag.toString())
.detail("TotalTags", req.totalTags)
.detail("StartVersion", req.startVersion)
.detail("EndVersion", req.endVersion.present() ? req.endVersion.get() : -1)
.detail("LogEpoch", req.recruitedEpoch)
.detail("BackupEpoch", req.backupEpoch);
try {
addActor.send(monitorBackupKeyOrPullData(&self));
addActor.send(checkRemoved(db, req.recruitedEpoch, &self));
addActor.send(waitFailureServer(interf.waitFailure.getFuture()));
if (req.recruitedEpoch == req.backupEpoch && req.routerTag.id == 0) {
addActor.send(monitorAllWorkerStarted(&self));
addActor.send(monitorBackupProgress(&self));
}
state Future<Void> done = uploadData(&self);
// Check if backup key is present to avoid race between this check and
// noop pop as well as upload data: pop or skip upload before knowing
// there are backup keys. Set the "exitEarly" flag if needed.
bool present = wait(monitorBackupStartedKeyChanges(&self, true, false));
TraceEvent("BackupWorkerWaitKey", self.myId).detail("Present", present).detail("ExitEarly", self.exitEarly);
pull = self.exitEarly ? Void() : monitorBackupKeyOrPullData(&self, present);
done = self.exitEarly ? Void() : uploadData(&self);
loop choose {
when(wait(dbInfoChange)) {
dbInfoChange = db->onChange();
Reference<ILogSystem> ls = ILogSystem::fromServerDBInfo(self.myId, db->get(), true);
bool hasPseudoLocality = ls.isValid() && ls->hasPseudoLocality(tagLocalityBackup);
LogEpoch oldestBackupEpoch = 0;
if (hasPseudoLocality) {
self.logSystem.set(ls);
self.pop();
oldestBackupEpoch = ls->getOldestBackupEpoch();
self.oldestBackupEpoch = std::max(self.oldestBackupEpoch, ls->getOldestBackupEpoch());
}
TraceEvent("BackupWorkerLogSystem", self.myId)
.detail("HasBackupLocality", hasPseudoLocality)
.detail("OldestBackupEpoch", oldestBackupEpoch)
.detail("OldestBackupEpoch", self.oldestBackupEpoch)
.detail("Tag", self.tag.toString());
}
when(wait(done)) {
@ -688,9 +932,15 @@ ACTOR Future<Void> backupWorker(BackupInterface interf, InitializeBackupRequest
when(wait(error)) {}
}
} catch (Error& e) {
TraceEvent("BackupWorkerTerminated", self.myId).error(e, true);
if (e.code() != error_code_actor_cancelled && e.code() != error_code_worker_removed) {
throw;
state Error err = e;
if (e.code() == error_code_worker_removed) {
pull = Void(); // cancels pulling
self.stop();
wait(done);
}
TraceEvent("BackupWorkerTerminated", self.myId).error(err, true);
if (err.code() != error_code_actor_cancelled && err.code() != error_code_worker_removed) {
throw err;
}
}
return Void();

View File

@ -2052,8 +2052,17 @@ ACTOR Future<Void> clusterRecruitRemoteFromConfiguration( ClusterControllerData*
void clusterRegisterMaster( ClusterControllerData* self, RegisterMasterRequest const& req ) {
req.reply.send( Void() );
TraceEvent("MasterRegistrationReceived", self->id).detail("MasterId", req.id).detail("Master", req.mi.toString()).detail("Tlogs", describe(req.logSystemConfig.tLogs)).detail("Resolvers", req.resolvers.size())
.detail("RecoveryState", (int)req.recoveryState).detail("RegistrationCount", req.registrationCount).detail("Proxies", req.proxies.size()).detail("RecoveryCount", req.recoveryCount).detail("Stalled", req.recoveryStalled);
TraceEvent("MasterRegistrationReceived", self->id)
.detail("MasterId", req.id)
.detail("Master", req.mi.toString())
.detail("Tlogs", describe(req.logSystemConfig.tLogs))
.detail("Resolvers", req.resolvers.size())
.detail("RecoveryState", (int)req.recoveryState)
.detail("RegistrationCount", req.registrationCount)
.detail("Proxies", req.proxies.size())
.detail("RecoveryCount", req.recoveryCount)
.detail("Stalled", req.recoveryStalled)
.detail("OldestBackupEpoch", req.logSystemConfig.oldestBackupEpoch);
//make sure the request comes from an active database
auto db = &self->db;

View File

@ -355,6 +355,7 @@ ServerKnobs::ServerKnobs(bool randomize, ClientKnobs* clientKnobs, bool isSimula
init( PROVISIONAL_START_DELAY, 1.0 );
init( PROVISIONAL_MAX_DELAY, 60.0 );
init( PROVISIONAL_DELAY_GROWTH, 1.5 );
init( SECONDS_BEFORE_RECRUIT_BACKUP_WORKER, 4.0 );
// Resolver
init( SAMPLE_OFFSET_PER_KEY, 100 );
@ -569,6 +570,7 @@ ServerKnobs::ServerKnobs(bool randomize, ClientKnobs* clientKnobs, bool isSimula
init( FASTRESTORE_APPLYING_PARALLELISM, 100 ); if( randomize ) { FASTRESTORE_APPLYING_PARALLELISM = deterministicRandom()->random01() * 10 + 1; }
init( FASTRESTORE_MONITOR_LEADER_DELAY, 5 ); if( randomize ) { FASTRESTORE_MONITOR_LEADER_DELAY = deterministicRandom()->random01() * 100; }
init( FASTRESTORE_STRAGGLER_THRESHOLD_SECONDS, 60 ); if( randomize && BUGGIFY ) { FASTRESTORE_STRAGGLER_THRESHOLD_SECONDS = deterministicRandom()->random01() * 240 + 10; }
init( FASTRESTORE_USE_PARTITIONED_LOGS, true );
init( FASTRESTORE_TRACK_REQUEST_LATENCY, true ); if( randomize && BUGGIFY ) { FASTRESTORE_TRACK_REQUEST_LATENCY = false; }
init( FASTRESTORE_TRACK_LOADER_SEND_REQUESTS, false ); if( randomize && BUGGIFY ) { FASTRESTORE_TRACK_LOADER_SEND_REQUESTS = true; }
init( FASTRESTORE_MEMORY_THRESHOLD_MB_SOFT, 6144 ); if( randomize && BUGGIFY ) { FASTRESTORE_MEMORY_THRESHOLD_MB_SOFT = 1; }

View File

@ -292,6 +292,7 @@ public:
double PROVISIONAL_START_DELAY;
double PROVISIONAL_DELAY_GROWTH;
double PROVISIONAL_MAX_DELAY;
double SECONDS_BEFORE_RECRUIT_BACKUP_WORKER;
// Resolver
int64_t KEY_BYTES_PER_SAMPLE;
@ -511,6 +512,7 @@ public:
int64_t FASTRESTORE_APPLYING_PARALLELISM; // number of outstanding txns writing to dest. DB
int64_t FASTRESTORE_MONITOR_LEADER_DELAY;
int64_t FASTRESTORE_STRAGGLER_THRESHOLD_SECONDS;
bool FASTRESTORE_USE_PARTITIONED_LOGS;
bool FASTRESTORE_TRACK_REQUEST_LATENCY; // true to track reply latency of each request in a request batch
bool FASTRESTORE_TRACK_LOADER_SEND_REQUESTS; // track requests of load send mutations to appliers?
int64_t FASTRESTORE_MEMORY_THRESHOLD_MB_SOFT; // threshold when pipelined actors should be delayed

View File

@ -36,7 +36,7 @@ struct DBCoreState;
struct TLogSet;
struct CoreTLogSet;
// The set of tLog servers and logRouters for a log tag
// The set of tLog servers, logRouters and backupWorkers for a log tag
class LogSet : NonCopyable, public ReferenceCounted<LogSet> {
public:
std::vector<Reference<AsyncVar<OptionalInterface<TLogInterface>>>> logServers;
@ -721,7 +721,8 @@ struct ILogSystem {
// Call only on an ILogSystem obtained from recoverAndEndEpoch()
// Returns the first unreadable version number of the recovered epoch (i.e. message version numbers < (get_end(), 0) will be readable)
virtual Version getStartVersion() const = 0; // Returns the start version of current epoch.
// Returns the start version of current epoch for backup workers.
virtual Version getBackupStartVersion() const = 0;
struct EpochTagsVersionsInfo {
int32_t logRouterTags; // Number of log router tags.
@ -784,6 +785,7 @@ struct ILogSystem {
virtual bool removeBackupWorker(const BackupWorkerDoneRequest& req) = 0;
virtual LogEpoch getOldestBackupEpoch() const = 0;
virtual void setOldestBackupEpoch(LogEpoch epoch) = 0;
};
struct LengthPrefixedStringRef {

View File

@ -100,9 +100,11 @@ ACTOR Future<Void> restoreApplierCore(RestoreApplierInterface applierInterf, int
}
// The actor may be invovked multiple times and executed async.
// No race condition as long as we do not wait or yield when operate the shared data.
// Multiple such actors can run on different fileIDs, because mutations in different files belong to different versions;
// Only one actor can process mutations from the same file
// No race condition as long as we do not wait or yield when operate the shared
// data. Multiple such actors can run on different fileIDs.
// Different files may contain mutations of the same commit versions, but with
// different subsequence number.
// Only one actor can process mutations from the same file.
ACTOR static Future<Void> handleSendMutationVectorRequest(RestoreSendVersionedMutationsRequest req,
Reference<RestoreApplierData> self) {
state Reference<ApplierBatchData> batchData = self->batch[req.batchIndex];
@ -126,21 +128,21 @@ ACTOR static Future<Void> handleSendMutationVectorRequest(RestoreSendVersionedMu
state bool isDuplicated = true;
if (curFilePos.get() == req.prevVersion) {
isDuplicated = false;
Version commitVersion = req.version;
const Version commitVersion = req.version;
uint16_t numVersionStampedKV = 0;
MutationsVec mutations(req.mutations);
// Sanity check: mutations in range file is in [beginVersion, endVersion);
// mutations in log file is in [beginVersion, endVersion], both inclusive.
ASSERT_WE_THINK(commitVersion >= req.asset.beginVersion);
ASSERT(commitVersion >= req.asset.beginVersion);
// Loader sends the endVersion to ensure all useful versions are sent
ASSERT_WE_THINK(commitVersion <= req.asset.endVersion);
ASSERT(commitVersion <= req.asset.endVersion);
ASSERT(req.mutations.size() == req.subs.size());
for (int mIndex = 0; mIndex < mutations.size(); mIndex++) {
MutationRef mutation = mutations[mIndex];
for (int mIndex = 0; mIndex < req.mutations.size(); mIndex++) {
const MutationRef& mutation = req.mutations[mIndex];
const LogMessageVersion mutationVersion(commitVersion, req.subs[mIndex]);
TraceEvent(SevFRMutationInfo, "FastRestoreApplierPhaseReceiveMutations", self->id())
.detail("ApplierNode", self->id())
.detail("RestoreAsset", req.asset.toString())
.detail("Version", commitVersion)
.detail("Version", mutationVersion.toString())
.detail("Index", mIndex)
.detail("MutationReceived", mutation.toString());
batchData->counters.receivedBytes += mutation.totalSize();
@ -159,10 +161,10 @@ ACTOR static Future<Void> handleSendMutationVectorRequest(RestoreSendVersionedMu
// Note: Log and range mutations may be delivered out of order. Can we handle it?
if (mutation.type == MutationRef::SetVersionstampedKey ||
mutation.type == MutationRef::SetVersionstampedValue) {
batchData->addVersionStampedKV(mutation, commitVersion, numVersionStampedKV);
batchData->addVersionStampedKV(mutation, mutationVersion, numVersionStampedKV);
numVersionStampedKV++;
} else {
batchData->addMutation(mutation, commitVersion);
batchData->addMutation(mutation, mutationVersion);
}
}
curFilePos.set(req.version);
@ -202,8 +204,8 @@ ACTOR static Future<Void> getAndComputeStagingKeys(
std::map<Key, std::map<Key, StagingKey>::iterator> incompleteStagingKeys, Database cx, UID applierID) {
state Reference<ReadYourWritesTransaction> tr(new ReadYourWritesTransaction(cx));
state std::vector<Future<Optional<Value>>> fValues;
state int i = 0;
state int retries = 0;
TraceEvent("FastRestoreApplierGetAndComputeStagingKeysStart", applierID)
.detail("GetKeys", incompleteStagingKeys.size());
loop {
@ -228,7 +230,7 @@ ACTOR static Future<Void> getAndComputeStagingKeys(
}
ASSERT(fValues.size() == incompleteStagingKeys.size());
i = 0;
int i = 0;
for (auto& key : incompleteStagingKeys) {
if (!fValues[i].get().present()) {
TraceEvent(SevWarnAlways, "FastRestoreApplierGetAndComputeStagingKeysUnhandledError")
@ -237,11 +239,9 @@ ACTOR static Future<Void> getAndComputeStagingKeys(
.detail("PendingMutations", key.second->second.pendingMutations.size())
.detail("StagingKeyType", (int)key.second->second.type);
for (auto& vm : key.second->second.pendingMutations) {
for (auto& m : vm.second) {
TraceEvent(SevWarnAlways, "FastRestoreApplierGetAndComputeStagingKeysUnhandledError")
.detail("PendingMutationVersion", vm.first)
.detail("PendingMutation", m.toString());
}
TraceEvent(SevWarnAlways, "FastRestoreApplierGetAndComputeStagingKeysUnhandledError")
.detail("PendingMutationVersion", vm.first.toString())
.detail("PendingMutation", vm.second.toString());
}
key.second->second.precomputeResult();
i++;
@ -250,7 +250,7 @@ ACTOR static Future<Void> getAndComputeStagingKeys(
// The key's version ideally should be the most recently committed version.
// But as long as it is > 1 and less than the start version of the version batch, it is the same result.
MutationRef m(MutationRef::SetValue, key.first, fValues[i].get().get());
key.second->second.add(m, (Version)1);
key.second->second.add(m, LogMessageVersion(1));
key.second->second.precomputeResult();
i++;
}

View File

@ -51,131 +51,117 @@ struct StagingKey {
Key key; // TODO: Maybe not needed?
Value val;
MutationRef::Type type; // set or clear
Version version; // largest version of set or clear for the key
std::map<Version, MutationsVec> pendingMutations; // mutations not set or clear type
LogMessageVersion version; // largest version of set or clear for the key
std::map<LogMessageVersion, Standalone<MutationRef>> pendingMutations; // mutations not set or clear type
explicit StagingKey() : version(0), type(MutationRef::MAX_ATOMIC_OP) {}
// Add mutation m at newVersion to stagingKey
// Assume: SetVersionstampedKey and SetVersionstampedValue have been converted to set
void add(const MutationRef& m, Version newVersion) {
void add(const MutationRef& m, LogMessageVersion newVersion) {
ASSERT(m.type != MutationRef::SetVersionstampedKey && m.type != MutationRef::SetVersionstampedValue);
if (version < newVersion) {
if (m.type == MutationRef::SetValue || m.type == MutationRef::ClearRange) {
if (debugMutation("StagingKeyAdd", newVersion.version, m)) {
TraceEvent("StagingKeyAdd")
.detail("Version", version.toString())
.detail("NewVersion", newVersion.toString())
.detail("Mutation", m.toString());
}
if (version == newVersion) {
// This could happen because the same mutation can be present in
// overlapping mutation logs, because new TLogs can copy mutations
// from old generation TLogs (or backup worker is recruited without
// knowning previously saved progress).
ASSERT(type == m.type && key == m.param1 && val == m.param2);
TraceEvent("SameVersion").detail("Version", version.toString()).detail("Mutation", m.toString());
return;
}
// newVersion can be smaller than version as different loaders can send
// mutations out of order.
if (m.type == MutationRef::SetValue || m.type == MutationRef::ClearRange) {
if (version < newVersion) {
key = m.param1;
val = m.param2;
type = (MutationRef::Type)m.type;
version = newVersion;
}
} else {
auto it = pendingMutations.find(newVersion);
if (it == pendingMutations.end()) {
pendingMutations.emplace(newVersion, m);
} else {
if (pendingMutations.find(newVersion) == pendingMutations.end()) {
pendingMutations.emplace(newVersion, MutationsVec());
}
// TODO: Do we really need deep copy?
MutationsVec& mutations = pendingMutations[newVersion];
mutations.push_back_deep(mutations.arena(), m);
// Duplicated mutation ignored.
TraceEvent("SameVersion")
.detail("Version", version.toString())
.detail("Mutation", m.toString())
.detail("NewVersion", newVersion.toString());
ASSERT(it->second.type == m.type && it->second.param1 == m.param1 && it->second.param2 == m.param2);
}
} else if (version == newVersion) { // Sanity check
TraceEvent("FastRestoreApplierStagingKeyMutationAtSameVersion")
.detail("Version", newVersion)
.detail("NewMutation", m.toString())
.detail("ExistingKeyType", typeString[type]);
if (m.type == MutationRef::SetValue) {
if (type == MutationRef::SetValue) {
if (m.param2 != val) {
TraceEvent(SevError, "FastRestoreApplierStagingKeyMutationAtSameVersionUnhandled")
.detail("Version", newVersion)
.detail("NewMutation", m.toString())
.detail("ExistingKeyType", typeString[type])
.detail("ExitingKeyValue", val)
.detail("Investigate",
"Why would backup have two sets with different value at same version");
} // else {} Backup has duplicate set at the same version
} else {
TraceEvent(SevWarnAlways, "FastRestoreApplierStagingKeyMutationAtSameVersionOverride")
.detail("Version", newVersion)
.detail("NewMutation", m.toString())
.detail("ExistingKeyType", typeString[type])
.detail("ExitingKeyValue", val);
type = (MutationRef::Type)m.type;
val = m.param2;
}
} else if (m.type == MutationRef::ClearRange) {
TraceEvent(SevWarnAlways, "FastRestoreApplierStagingKeyMutationAtSameVersionSkipped")
.detail("Version", newVersion)
.detail("NewMutation", m.toString())
.detail("ExistingKeyType", typeString[type])
.detail("ExitingKeyValue", val);
}
} // else input mutation is old and can be ignored
}
}
// Precompute the final value of the key.
// TODO: Look at the last LogMessageVersion, if it set or clear, we can ignore the rest of versions.
void precomputeResult() {
TraceEvent(SevDebug, "FastRestoreApplierPrecomputeResult")
.detail("Key", key)
.detail("Version", version)
.detail("LargestPendingVersion", (pendingMutations.empty() ? -1 : pendingMutations.rbegin()->first));
std::map<Version, MutationsVec>::iterator lb = pendingMutations.lower_bound(version);
.detail("Version", version.toString())
.detail("LargestPendingVersion",
(pendingMutations.empty() ? "[none]" : pendingMutations.rbegin()->first.toString()));
std::map<LogMessageVersion, Standalone<MutationRef>>::iterator lb = pendingMutations.lower_bound(version);
if (lb == pendingMutations.end()) {
return;
}
if (lb->first == version) {
// Sanity check mutations at version are either atomicOps which can be ignored or the same value as buffered
for (int i = 0; i < lb->second.size(); i++) {
MutationRef m = lb->second[i];
if (m.type == MutationRef::SetValue || m.type == MutationRef::ClearRange) {
if (std::tie(type, key, val) != std::tie(m.type, m.param1, m.param2)) {
TraceEvent(SevError, "FastRestoreApplierPrecomputeResultUnhandledSituation")
.detail("BufferedType", typeString[type])
.detail("PendingType", typeString[m.type])
.detail("BufferedVal", val.toString())
.detail("PendingVal", m.param2.toString());
}
MutationRef m = lb->second;
if (m.type == MutationRef::SetValue || m.type == MutationRef::ClearRange) {
if (std::tie(type, key, val) != std::tie(m.type, m.param1, m.param2)) {
TraceEvent(SevError, "FastRestoreApplierPrecomputeResultUnhandledSituation")
.detail("BufferedType", typeString[type])
.detail("PendingType", typeString[m.type])
.detail("BufferedVal", val.toString())
.detail("PendingVal", m.param2.toString());
}
}
lb++;
}
while (lb != pendingMutations.end()) {
if (lb->first == version) {
lb++;
continue;
}
for (auto& mutation : lb->second) {
if (type == MutationRef::CompareAndClear) { // Special atomicOp
Arena arena;
Optional<ValueRef> retVal = doCompareAndClear(val, mutation.param2, arena);
if (!retVal.present()) {
val = key;
type = MutationRef::ClearRange;
} // else no-op
} else if (isAtomicOp((MutationRef::Type)mutation.type)) {
Optional<StringRef> inputVal;
if (hasBaseValue()) {
inputVal = val;
}
val = applyAtomicOp(inputVal, mutation.param2, (MutationRef::Type)mutation.type);
type = MutationRef::SetValue; // Precomputed result should be set to DB.
} else if (mutation.type == MutationRef::SetValue || mutation.type == MutationRef::ClearRange) {
type = MutationRef::SetValue; // Precomputed result should be set to DB.
TraceEvent(SevError, "FastRestoreApplierPrecomputeResultUnexpectedSet")
.detail("Type", typeString[mutation.type])
.detail("Version", lb->first);
} else {
TraceEvent(SevWarnAlways, "FastRestoreApplierPrecomputeResultSkipUnexpectedBackupMutation")
.detail("Type", typeString[mutation.type])
.detail("Version", lb->first);
for (; lb != pendingMutations.end(); lb++) {
MutationRef mutation = lb->second;
if (type == MutationRef::CompareAndClear) { // Special atomicOp
Arena arena;
Optional<ValueRef> retVal = doCompareAndClear(val, mutation.param2, arena);
if (!retVal.present()) {
val = key;
type = MutationRef::ClearRange;
} // else no-op
} else if (isAtomicOp((MutationRef::Type)mutation.type)) {
Optional<StringRef> inputVal;
if (hasBaseValue()) {
inputVal = val;
}
val = applyAtomicOp(inputVal, mutation.param2, (MutationRef::Type)mutation.type);
type = MutationRef::SetValue; // Precomputed result should be set to DB.
} else if (mutation.type == MutationRef::SetValue || mutation.type == MutationRef::ClearRange) {
type = MutationRef::SetValue; // Precomputed result should be set to DB.
TraceEvent(SevError, "FastRestoreApplierPrecomputeResultUnexpectedSet")
.detail("MutationType", typeString[mutation.type])
.detail("Version", lb->first.toString());
} else {
TraceEvent(SevWarnAlways, "FastRestoreApplierPrecomputeResultSkipUnexpectedBackupMutation")
.detail("MutationType", typeString[mutation.type])
.detail("Version", lb->first.toString());
}
version = lb->first;
lb++;
}
}
// Does the key has at least 1 set or clear mutation to get the base value
bool hasBaseValue() {
if (version > 0) {
if (version.version > 0) {
ASSERT(type == MutationRef::SetValue || type == MutationRef::ClearRange);
}
return version > 0;
return version.version > 0;
}
// Has all pendingMutations been pre-applied to the val?
@ -191,9 +177,9 @@ struct StagingKey {
// Range mutations should be applied both to the destination DB and to the StagingKeys
struct StagingKeyRange {
Standalone<MutationRef> mutation;
Version version;
LogMessageVersion version;
explicit StagingKeyRange(MutationRef m, Version newVersion) : mutation(m), version(newVersion) {}
explicit StagingKeyRange(MutationRef m, LogMessageVersion newVersion) : mutation(m), version(newVersion) {}
bool operator<(const StagingKeyRange& rhs) const {
return std::tie(version, mutation.type, mutation.param1, mutation.param2) <
@ -263,7 +249,7 @@ struct ApplierBatchData : public ReferenceCounted<ApplierBatchData> {
}
~ApplierBatchData() = default;
void addMutation(MutationRef m, Version ver) {
void addMutation(MutationRef m, LogMessageVersion ver) {
if (!isRangeMutation(m)) {
auto item = stagingKeys.emplace(m.param1, StagingKey());
item.first->second.add(m, ver);
@ -272,20 +258,20 @@ struct ApplierBatchData : public ReferenceCounted<ApplierBatchData> {
}
}
void addVersionStampedKV(MutationRef m, Version ver, uint16_t numVersionStampedKV) {
void addVersionStampedKV(MutationRef m, LogMessageVersion ver, uint16_t numVersionStampedKV) {
if (m.type == MutationRef::SetVersionstampedKey) {
// Assume transactionNumber = 0 does not affect result
TraceEvent(SevDebug, "FastRestoreApplierAddMutation")
.detail("MutationType", typeString[m.type])
.detail("FakedTransactionNumber", numVersionStampedKV);
transformVersionstampMutation(m, &MutationRef::param1, ver, numVersionStampedKV);
transformVersionstampMutation(m, &MutationRef::param1, ver.version, numVersionStampedKV);
addMutation(m, ver);
} else if (m.type == MutationRef::SetVersionstampedValue) {
// Assume transactionNumber = 0 does not affect result
TraceEvent(SevDebug, "FastRestoreApplierAddMutation")
.detail("MutationType", typeString[m.type])
.detail("FakedTransactionNumber", numVersionStampedKV);
transformVersionstampMutation(m, &MutationRef::param2, ver, numVersionStampedKV);
transformVersionstampMutation(m, &MutationRef::param2, ver.version, numVersionStampedKV);
addMutation(m, ver);
} else {
ASSERT(false);
@ -298,8 +284,8 @@ struct ApplierBatchData : public ReferenceCounted<ApplierBatchData> {
if (!stagingKey.second.hasPrecomputed()) {
TraceEvent("FastRestoreApplierAllKeysPrecomputedFalse")
.detail("Key", stagingKey.first)
.detail("BufferedVersion", stagingKey.second.version)
.detail("MaxPendingVersion", stagingKey.second.pendingMutations.rbegin()->first);
.detail("BufferedVersion", stagingKey.second.version.toString())
.detail("MaxPendingVersion", stagingKey.second.pendingMutations.rbegin()->first.toString());
return false;
}
}
@ -320,20 +306,17 @@ struct ApplierBatchData : public ReferenceCounted<ApplierBatchData> {
}
bool isKVOpsSorted() {
bool ret = true;
auto prev = kvOps.begin();
for (auto it = kvOps.begin(); it != kvOps.end(); ++it) {
if (prev->first > it->first) {
ret = false;
break;
return false;
}
prev = it;
}
return ret;
return true;
}
bool allOpsAreKnown() {
bool ret = true;
for (auto it = kvOps.begin(); it != kvOps.end(); ++it) {
for (auto m = it->second.begin(); m != it->second.end(); ++m) {
if (m->type == MutationRef::SetValue || m->type == MutationRef::ClearRange ||
@ -341,11 +324,11 @@ struct ApplierBatchData : public ReferenceCounted<ApplierBatchData> {
continue;
else {
TraceEvent(SevError, "FastRestore").detail("UnknownMutationType", m->type);
ret = false;
return false;
}
}
}
return ret;
return true;
}
};

View File

@ -23,15 +23,15 @@
#include "fdbserver/RestoreCommon.actor.h"
// Backup agent header
#include "fdbclient/BackupAgent.actor.h"
#include "fdbclient/BackupContainer.h"
#include "fdbclient/KeyBackedTypes.h"
#include "fdbclient/ManagementAPI.actor.h"
#include "fdbclient/MutationList.h"
#include "fdbclient/NativeAPI.actor.h"
#include "fdbclient/SystemData.h"
// Backup agent header
#include "fdbclient/BackupAgent.actor.h"
//#include "FileBackupAgent.h"
#include "fdbclient/ManagementAPI.actor.h"
#include "fdbclient/MutationList.h"
#include "fdbclient/BackupContainer.h"
#include "flow/actorcompiler.h" // This must be the last #include.
// Split RestoreConfigFR defined in FileBackupAgent.actor.cpp to declaration in Restore.actor.h and implementation in
@ -296,38 +296,6 @@ std::string RestoreConfigFR::toString() {
// The implementation of parallelFileRestore is copied from FileBackupAgent.actor.cpp
// parallelFileRestore is copied from FileBackupAgent.actor.cpp for the same reason as RestoreConfigFR is copied
namespace parallelFileRestore {
// Helper class for reading restore data from a buffer and throwing the right errors.
struct StringRefReader {
StringRefReader(StringRef s = StringRef(), Error e = Error()) : rptr(s.begin()), end(s.end()), failure_error(e) {}
// Return remainder of data as a StringRef
StringRef remainder() { return StringRef(rptr, end - rptr); }
// Return a pointer to len bytes at the current read position and advance read pos
const uint8_t* consume(unsigned int len) {
if (rptr == end && len != 0) throw end_of_stream();
const uint8_t* p = rptr;
rptr += len;
if (rptr > end) throw failure_error;
return p;
}
// Return a T from the current read position and advance read pos
template <typename T>
const T consume() {
return *(const T*)consume(sizeof(T));
}
// Functions for consuming big endian (network byte order) integers.
// Consumes a big endian number, swaps it to little endian, and returns it.
int32_t consumeNetworkInt32() { return (int32_t)bigEndian32((uint32_t)consume<int32_t>()); }
uint32_t consumeNetworkUInt32() { return bigEndian32(consume<uint32_t>()); }
bool eof() { return rptr == end; }
const uint8_t *rptr, *end;
Error failure_error;
};
ACTOR Future<Standalone<VectorRef<KeyValueRef>>> decodeRangeFileBlock(Reference<IAsyncFile> file, int64_t offset,
int len) {
@ -336,7 +304,7 @@ ACTOR Future<Standalone<VectorRef<KeyValueRef>>> decodeRangeFileBlock(Reference<
if (rLen != len) throw restore_bad_read();
Standalone<VectorRef<KeyValueRef>> results({}, buf.arena());
state parallelFileRestore::StringRefReader reader(buf, restore_corrupted_data());
state StringRefReader reader(buf, restore_corrupted_data());
try {
// Read header, currently only decoding version 1001
@ -393,11 +361,11 @@ ACTOR Future<Standalone<VectorRef<KeyValueRef>>> decodeLogFileBlock(Reference<IA
if (rLen != len) throw restore_bad_read();
Standalone<VectorRef<KeyValueRef>> results({}, buf.arena());
state parallelFileRestore::StringRefReader reader(buf, restore_corrupted_data());
state StringRefReader reader(buf, restore_corrupted_data());
try {
// Read header, currently only decoding version 2001
if (reader.consume<int32_t>() != 2001) throw restore_unsupported_file_version();
// Read header, currently only decoding version BACKUP_AGENT_MLOG_VERSION
if (reader.consume<int32_t>() != BACKUP_AGENT_MLOG_VERSION) throw restore_unsupported_file_version();
// Read k/v pairs. Block ends either at end of last value exactly or with 0xFF as first key len byte.
while (1) {

View File

@ -188,6 +188,7 @@ struct RestoreFileFR {
int64_t cursor; // The start block location to be restored. All blocks before cursor have been scheduled to load and
// restore
int fileIndex; // index of backup file. Must be identical per file.
int partitionId = -1; // Partition ID (Log Router Tag ID) for mutation files.
Tuple pack() const {
return Tuple()
@ -199,7 +200,8 @@ struct RestoreFileFR {
.append(endVersion)
.append(beginVersion)
.append(cursor)
.append(fileIndex);
.append(fileIndex)
.append(partitionId);
}
static RestoreFileFR unpack(Tuple const& t) {
RestoreFileFR r;
@ -213,6 +215,7 @@ struct RestoreFileFR {
r.beginVersion = t.getInt(i++);
r.cursor = t.getInt(i++);
r.fileIndex = t.getInt(i++);
r.partitionId = t.getInt(i++);
return r;
}
@ -225,18 +228,21 @@ struct RestoreFileFR {
: version(invalidVersion), isRange(false), blockSize(0), fileSize(0), endVersion(invalidVersion),
beginVersion(invalidVersion), cursor(0), fileIndex(0) {}
RestoreFileFR(Version version, std::string fileName, bool isRange, int64_t blockSize, int64_t fileSize,
Version endVersion, Version beginVersion)
: version(version), fileName(fileName), isRange(isRange), blockSize(blockSize), fileSize(fileSize),
endVersion(endVersion), beginVersion(beginVersion), cursor(0), fileIndex(0) {}
explicit RestoreFileFR(const RangeFile& f)
: version(f.version), fileName(f.fileName), isRange(true), blockSize(f.blockSize), fileSize(f.fileSize),
endVersion(f.version), beginVersion(f.version), cursor(0), fileIndex(0) {}
explicit RestoreFileFR(const LogFile& f)
: version(f.beginVersion), fileName(f.fileName), isRange(false), blockSize(f.blockSize), fileSize(f.fileSize),
endVersion(f.endVersion), beginVersion(f.beginVersion), cursor(0), fileIndex(0), partitionId(f.tagId) {}
std::string toString() const {
std::stringstream ss;
ss << "version:" << std::to_string(version) << " fileName:" << fileName
<< " isRange:" << std::to_string(isRange) << " blockSize:" << std::to_string(blockSize)
<< " fileSize:" << std::to_string(fileSize) << " endVersion:" << std::to_string(endVersion)
<< " beginVersion:" << std::to_string(beginVersion) << " cursor:" << std::to_string(cursor)
<< " fileIndex:" << std::to_string(fileIndex);
ss << "version:" << version << " fileName:" << fileName
<< " isRange:" << isRange << " blockSize:" << blockSize
<< " fileSize:" << fileSize << " endVersion:" << endVersion
<< " beginVersion:" << beginVersion << " cursor:" << cursor
<< " fileIndex:" << fileIndex << " partitionId:" << partitionId;
return ss.str();
}
};

View File

@ -3,7 +3,7 @@
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2018 Apple Inc. and the FoundationDB project authors
* Copyright 2013-2020 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.
@ -23,6 +23,7 @@
#include "fdbclient/BackupContainer.h"
#include "fdbserver/RestoreLoader.actor.h"
#include "fdbserver/RestoreRoleCommon.actor.h"
#include "flow/actorcompiler.h" // This must be the last #include.
@ -140,6 +141,92 @@ void handleRestoreSysInfoRequest(const RestoreSysInfoRequest& req, Reference<Res
req.reply.send(RestoreCommonReply(self->id()));
}
// Parse a data block in a partitioned mutation log file and store mutations
// into "kvOpsIter" and samples into "samplesIter".
ACTOR static Future<Void> _parsePartitionedLogFileOnLoader(
NotifiedVersion* processedFileOffset, std::map<LoadingParam, VersionedMutationsMap>::iterator kvOpsIter,
std::map<LoadingParam, MutationsVec>::iterator samplesIter, Reference<IBackupContainer> bc, RestoreAsset asset) {
state Standalone<StringRef> buf = makeString(asset.len);
state Reference<IAsyncFile> file = wait(bc->readFile(asset.filename));
int rLen = wait(file->read(mutateString(buf), asset.len, asset.offset));
if (rLen != asset.len) throw restore_bad_read();
TraceEvent("FastRestore")
.detail("DecodingLogFile", asset.filename)
.detail("Offset", asset.offset)
.detail("Length", asset.len);
// Ensure data blocks in the same file are processed in order
wait(processedFileOffset->whenAtLeast(asset.offset));
ASSERT(processedFileOffset->get() == asset.offset);
StringRefReader reader(buf, restore_corrupted_data());
try {
// Read block header
if (reader.consume<int32_t>() != PARTITIONED_MLOG_VERSION) throw restore_unsupported_file_version();
VersionedMutationsMap& kvOps = kvOpsIter->second;
while (1) {
// If eof reached or first key len bytes is 0xFF then end of block was reached.
if (reader.eof() || *reader.rptr == 0xFF) break;
// Deserialize messages written in saveMutationsToFile().
LogMessageVersion msgVersion;
msgVersion.version = reader.consumeNetworkUInt64();
msgVersion.sub = reader.consumeNetworkUInt32();
int msgSize = reader.consumeNetworkInt32();
const uint8_t* message = reader.consume(msgSize);
// Skip mutations out of the version range
if (!asset.isInVersionRange(msgVersion.version)) continue;
VersionedMutationsMap::iterator it;
bool inserted;
std::tie(it, inserted) = kvOps.emplace(msgVersion, MutationsVec());
ASSERT(inserted);
ArenaReader rd(buf.arena(), StringRef(message, msgSize), AssumeVersion(currentProtocolVersion));
MutationRef mutation;
rd >> mutation;
// Should this mutation be skipped?
if (mutation.param1 >= asset.range.end ||
(isRangeMutation(mutation) && mutation.param2 < asset.range.begin) ||
(!isRangeMutation(mutation) && mutation.param1 < asset.range.begin)) {
continue;
}
// Only apply mutation within the asset.range
if (isRangeMutation(mutation)) {
mutation.param1 = mutation.param1 >= asset.range.begin ? mutation.param1 : asset.range.begin;
mutation.param2 = mutation.param2 < asset.range.end ? mutation.param2 : asset.range.end;
}
TraceEvent(SevFRMutationInfo, "FastRestore_VerboseDebug")
.detail("CommitVersion", msgVersion.toString())
.detail("ParsedMutation", mutation.toString());
it->second.push_back_deep(it->second.arena(), mutation);
// Sampling (FASTRESTORE_SAMPLING_PERCENT%) data
if (deterministicRandom()->random01() * 100 < SERVER_KNOBS->FASTRESTORE_SAMPLING_PERCENT) {
samplesIter->second.push_back_deep(samplesIter->second.arena(), mutation);
}
}
// Make sure any remaining bytes in the block are 0xFF
for (auto b : reader.remainder()) {
if (b != 0xFF) throw restore_corrupted_data_padding();
}
} catch (Error& e) {
TraceEvent(SevWarn, "FileRestoreCorruptLogFileBlock")
.error(e)
.detail("Filename", file->getFilename())
.detail("BlockOffset", asset.offset)
.detail("BlockLen", asset.len);
throw;
}
processedFileOffset->set(asset.offset + asset.len);
return Void();
}
ACTOR Future<Void> _processLoadingParam(LoadingParam param, Reference<LoaderBatchData> batchData, UID loaderID,
Reference<IBackupContainer> bc) {
// Temporary data structure for parsing log files into (version, <K, V, mutationType>)
@ -155,15 +242,15 @@ ACTOR Future<Void> _processLoadingParam(LoadingParam param, Reference<LoaderBatc
// Q: How to record the param's fields inside LoadingParam Refer to storageMetrics
TraceEvent("FastRestoreLoaderProcessLoadingParam", loaderID).detail("LoadingParam", param.toString());
ASSERT(param.blockSize > 0);
ASSERT(param.asset.offset % param.blockSize == 0); // Parse file must be at block bondary.
ASSERT(param.asset.offset % param.blockSize == 0); // Parse file must be at block boundary.
ASSERT(batchData->kvOpsPerLP.find(param) == batchData->kvOpsPerLP.end());
// NOTE: map's iterator is guaranteed to be stable, but pointer may not.
// state VersionedMutationsMap* kvOps = &self->kvOpsPerLP[param];
batchData->kvOpsPerLP.emplace(param, VersionedMutationsMap());
batchData->sampleMutations.emplace(param, MutationsVec());
kvOpsPerLPIter = batchData->kvOpsPerLP.find(param);
samplesIter = batchData->sampleMutations.find(param);
bool inserted;
std::tie(kvOpsPerLPIter, inserted) = batchData->kvOpsPerLP.emplace(param, VersionedMutationsMap());
ASSERT(inserted);
std::tie(samplesIter, inserted) = batchData->sampleMutations.emplace(param, MutationsVec());
ASSERT(inserted);
for (int64_t j = param.asset.offset; j < param.asset.len; j += param.blockSize) {
RestoreAsset subAsset = param.asset;
@ -174,13 +261,18 @@ ACTOR Future<Void> _processLoadingParam(LoadingParam param, Reference<LoaderBatc
kvOpsPerLPIter, samplesIter, &batchData->counters, bc, param.rangeVersion.get(), subAsset));
} else {
// TODO: Sanity check the log file's range is overlapped with the restored version range
fileParserFutures.push_back(
_parseLogFileToMutationsOnLoader(&processedFileOffset, &mutationMap, &mutationPartMap, bc, subAsset));
if (param.isPartitionedLog()) {
fileParserFutures.push_back(_parsePartitionedLogFileOnLoader(&processedFileOffset, kvOpsPerLPIter,
samplesIter, bc, subAsset));
} else {
fileParserFutures.push_back(_parseLogFileToMutationsOnLoader(&processedFileOffset, &mutationMap,
&mutationPartMap, bc, subAsset));
}
}
}
wait(waitForAll(fileParserFutures));
if (!param.isRangeFile) {
if (!param.isRangeFile && !param.isPartitionedLog()) {
_parseSerializedMutation(kvOpsPerLPIter, &mutationMap, samplesIter, &batchData->counters, param.asset);
}
@ -211,7 +303,6 @@ ACTOR Future<Void> handleLoadFileRequest(RestoreLoadFileRequest req, Reference<R
.detail("BatchIndex", req.batchIndex)
.detail("ProcessLoadParam", req.param.toString());
ASSERT(batchData->sampleMutations.find(req.param) == batchData->sampleMutations.end());
batchData->processedFileParams[req.param] = Never(); // Ensure second exec. wait on _processLoadingParam()
batchData->processedFileParams[req.param] = _processLoadingParam(req.param, batchData, self->id(), self->bc);
isDuplicated = false;
} else {
@ -219,8 +310,9 @@ ACTOR Future<Void> handleLoadFileRequest(RestoreLoadFileRequest req, Reference<R
.detail("BatchIndex", req.batchIndex)
.detail("WaitOnProcessLoadParam", req.param.toString());
}
ASSERT(batchData->processedFileParams.find(req.param) != batchData->processedFileParams.end());
wait(batchData->processedFileParams[req.param]); // wait on the processing of the req.param.
auto it = batchData->processedFileParams.find(req.param);
ASSERT(it != batchData->processedFileParams.end());
wait(it->second); // wait on the processing of the req.param.
req.reply.send(RestoreLoadFileReply(req.param, batchData->sampleMutations[req.param], isDuplicated));
TraceEvent("FastRestoreLoaderPhaseLoadFileDone", self->id())
@ -235,7 +327,6 @@ ACTOR Future<Void> handleLoadFileRequest(RestoreLoadFileRequest req, Reference<R
ACTOR Future<Void> handleSendMutationsRequest(RestoreSendMutationsToAppliersRequest req,
Reference<RestoreLoaderData> self) {
state Reference<LoaderBatchData> batchData = self->batch[req.batchIndex];
state std::map<LoadingParam, VersionedMutationsMap>::iterator item = batchData->kvOpsPerLP.begin();
state Reference<LoaderBatchStatus> batchStatus = self->status[req.batchIndex];
state bool isDuplicated = true;
@ -285,11 +376,11 @@ ACTOR Future<Void> handleSendMutationsRequest(RestoreSendMutationsToAppliersRequ
if (!isDuplicated) {
vector<Future<Void>> fSendMutations;
batchData->rangeToApplier = req.rangeToApplier;
for (; item != batchData->kvOpsPerLP.end(); item++) {
if (item->first.isRangeFile == req.useRangeFile) {
for (auto& [loadParam, kvOps] : batchData->kvOpsPerLP) {
if (loadParam.isRangeFile == req.useRangeFile) {
// Send the parsed mutation to applier who will apply the mutation to DB
fSendMutations.push_back(sendMutationsToApplier(&item->second, req.batchIndex, item->first.asset,
item->first.isRangeFile, &batchData->rangeToApplier,
fSendMutations.push_back(sendMutationsToApplier(&kvOps, req.batchIndex, loadParam.asset,
loadParam.isRangeFile, &batchData->rangeToApplier,
&self->appliersInterf));
}
}
@ -331,41 +422,36 @@ ACTOR Future<Void> sendMutationsToApplier(VersionedMutationsMap* pkvOps, int bat
.detail("RestoreAsset", asset.toString());
// There should be no mutation at asset.endVersion version because it is exclusive
if (kvOps.find(asset.endVersion) != kvOps.end()) {
if (kvOps.lower_bound(LogMessageVersion(asset.endVersion)) != kvOps.end()) {
TraceEvent(SevError, "FastRestoreLoaderSendMutationToApplier")
.detail("BatchIndex", batchIndex)
.detail("RestoreAsset", asset.toString())
.detail("IsRangeFile", isRangeFile)
.detail("Data loss at version", asset.endVersion);
}
// Ensure there is a mutation request sent at endVersion, so that applier can advance its notifiedVersion
if (kvOps.find(asset.endVersion) == kvOps.end()) {
kvOps[asset.endVersion] = MutationsVec(); // Empty mutation vector will be handled by applier
} else {
// Ensure there is a mutation request sent at endVersion, so that applier can advance its notifiedVersion
kvOps[LogMessageVersion(asset.endVersion)] = MutationsVec(); // Empty mutation vector will be handled by applier
}
splitMutationIndex = 0;
kvCount = 0;
// applierMutationsBuffer is the mutation vector to be sent to each applier
// applierMutationsSize is buffered mutation vector size for each applier
state std::map<UID, MutationsVec> applierMutationsBuffer;
state std::map<UID, SubSequenceVec> applierSubsBuffer;
state std::map<UID, double> applierMutationsSize;
for (auto& applierID : applierIDs) {
applierMutationsBuffer[applierID] = MutationsVec();
applierSubsBuffer[applierID] = SubSequenceVec();
applierMutationsSize[applierID] = 0.0;
}
for (kvOp = kvOps.begin(); kvOp != kvOps.end(); kvOp++) {
// applierMutationsBuffer is the mutation vector to be sent to each applier
// applierMutationsSize is buffered mutation vector size for each applier
std::map<UID, MutationsVec> applierMutationsBuffer;
std::map<UID, double> applierMutationsSize;
for (auto& applierID : applierIDs) {
applierMutationsBuffer[applierID] = MutationsVec();
applierMutationsSize[applierID] = 0.0;
}
Version commitVersion = kvOp->first;
if (!(commitVersion >= asset.beginVersion && commitVersion <= asset.endVersion)) { // Debug purpose
TraceEvent(SevError, "FastRestore_SendMutationsToApplier")
.detail("CommitVersion", commitVersion)
.detail("RestoreAsset", asset.toString());
}
ASSERT(commitVersion >= asset.beginVersion);
ASSERT(commitVersion <= asset.endVersion); // endVersion is an empty commit to ensure progress
const LogMessageVersion& commitVersion = kvOp->first;
ASSERT(commitVersion.version >= asset.beginVersion);
ASSERT(commitVersion.version <= asset.endVersion); // endVersion is an empty commit to ensure progress
for (int mIndex = 0; mIndex < kvOp->second.size(); mIndex++) {
MutationRef kvm = kvOp->second[mIndex];
for (const MutationRef& kvm : kvOp->second) {
// Send the mutation to applier
if (isRangeMutation(kvm)) {
MutationsVec mvector;
@ -377,12 +463,27 @@ ACTOR Future<Void> sendMutationsToApplier(VersionedMutationsMap* pkvOps, int bat
nodeIDs.contents());
ASSERT(mvector.size() == nodeIDs.size());
if (debugMutation("RestoreLoader", commitVersion.version, kvm)) {
TraceEvent e("DebugSplit");
int i = 0;
for (auto& [key, uid] : *pRangeToApplier) {
e.detail(format("Range%d", i).c_str(), printable(key))
.detail(format("UID%d", i).c_str(), uid.toString());
i++;
}
}
for (splitMutationIndex = 0; splitMutationIndex < mvector.size(); splitMutationIndex++) {
MutationRef mutation = mvector[splitMutationIndex];
UID applierID = nodeIDs[splitMutationIndex];
// printf("SPLITTED MUTATION: %d: mutation:%s applierID:%s\n", splitMutationIndex,
// mutation.toString().c_str(), applierID.toString().c_str());
if (debugMutation("RestoreLoader", commitVersion.version, mutation)) {
TraceEvent("SplittedMutation")
.detail("Version", commitVersion.toString())
.detail("Mutation", mutation.toString());
}
applierMutationsBuffer[applierID].push_back_deep(applierMutationsBuffer[applierID].arena(), mutation);
applierSubsBuffer[applierID].push_back(applierSubsBuffer[applierID].arena(), commitVersion.sub);
applierMutationsSize[applierID] += mutation.expectedSize();
kvCount++;
@ -391,37 +492,47 @@ ACTOR Future<Void> sendMutationsToApplier(VersionedMutationsMap* pkvOps, int bat
std::map<Key, UID>::iterator itlow = pRangeToApplier->upper_bound(kvm.param1);
--itlow; // make sure itlow->first <= m.param1
ASSERT(itlow->first <= kvm.param1);
MutationRef mutation = kvm;
UID applierID = itlow->second;
// printf("KV--Applier: K:%s ApplierID:%s\n", kvm.param1.toString().c_str(),
// applierID.toString().c_str());
kvCount++;
applierMutationsBuffer[applierID].push_back_deep(applierMutationsBuffer[applierID].arena(), mutation);
applierMutationsSize[applierID] += mutation.expectedSize();
if (debugMutation("RestoreLoader", commitVersion.version, kvm)) {
TraceEvent("SendMutation")
.detail("Applier", applierID)
.detail("Version", commitVersion.toString())
.detail("Mutation", kvm.toString());
}
applierMutationsBuffer[applierID].push_back_deep(applierMutationsBuffer[applierID].arena(), kvm);
applierSubsBuffer[applierID].push_back(applierSubsBuffer[applierID].arena(), commitVersion.sub);
applierMutationsSize[applierID] += kvm.expectedSize();
}
} // Mutations at the same version
} // Mutations at the same LogMessageVersion
// TODO: Sanity check each asset has been received exactly once!
// Send the mutations to appliers for each version
for (auto& applierID : applierIDs) {
requests.push_back(std::make_pair(
applierID, RestoreSendVersionedMutationsRequest(batchIndex, asset, prevVersion, commitVersion,
isRangeFile, applierMutationsBuffer[applierID])));
// Batch same Version's mutations in one request. We could batch more by
// changing the version comparison below.
auto next = std::next(kvOp, 1);
if (next == kvOps.end() || commitVersion.version < next->first.version) {
// TODO: Sanity check each asset has been received exactly once!
// Send the mutations to appliers for each version
for (const UID& applierID : applierIDs) {
requests.emplace_back(applierID, RestoreSendVersionedMutationsRequest(
batchIndex, asset, prevVersion, commitVersion.version, isRangeFile,
applierMutationsBuffer[applierID], applierSubsBuffer[applierID]));
}
TraceEvent(SevDebug, "FastRestore_SendMutationToApplier")
.detail("PrevVersion", prevVersion)
.detail("CommitVersion", commitVersion.toString())
.detail("RestoreAsset", asset.toString());
ASSERT(prevVersion < commitVersion.version);
prevVersion = commitVersion.version;
wait(sendBatchRequests(&RestoreApplierInterface::sendMutationVector, *pApplierInterfaces, requests,
TaskPriority::RestoreLoaderSendMutations));
requests.clear();
for (auto& applierID : applierIDs) {
applierMutationsBuffer[applierID] = MutationsVec();
applierSubsBuffer[applierID] = SubSequenceVec();
applierMutationsSize[applierID] = 0.0;
}
}
TraceEvent(SevDebug, "FastRestore_SendMutationToApplier")
.detail("PrevVersion", prevVersion)
.detail("CommitVersion", commitVersion)
.detail("RestoreAsset", asset.toString());
ASSERT(prevVersion < commitVersion);
prevVersion = commitVersion;
// Tracking this request can be spammy
wait(sendBatchRequests(&RestoreApplierInterface::sendMutationVector, *pApplierInterfaces, requests,
TaskPriority::RestoreLoaderSendMutations,
SERVER_KNOBS->FASTRESTORE_TRACK_LOADER_SEND_REQUESTS));
requests.clear();
} // all versions of mutations in the same file
TraceEvent("FastRestore").detail("LoaderSendMutationOnAppliers", kvCount);
@ -436,8 +547,14 @@ void splitMutation(std::map<Key, UID>* pRangeToApplier, MutationRef m, Arena& mv
ASSERT(mvector.empty());
ASSERT(nodeIDs.empty());
// key range [m->param1, m->param2)
std::map<Standalone<KeyRef>, UID>::iterator itlow, itup; // we will return [itlow, itup)
std::map<Key, UID>::iterator itlow, itup; // we will return [itlow, itup)
itlow = pRangeToApplier->lower_bound(m.param1); // lower_bound returns the iterator that is >= m.param1
if (itlow == pRangeToApplier->end()) {
--itlow;
mvector.push_back_deep(mvector_arena, m);
nodeIDs.push_back(nodeIDs_arena, itlow->second);
return;
}
if (itlow->first > m.param1) {
if (itlow != pRangeToApplier->begin()) {
--itlow;
@ -447,7 +564,7 @@ void splitMutation(std::map<Key, UID>* pRangeToApplier, MutationRef m, Arena& mv
itup = pRangeToApplier->upper_bound(m.param2); // return rmap::end if no key is after m.param2.
ASSERT(itup == pRangeToApplier->end() || itup->first > m.param2);
std::map<Standalone<KeyRef>, UID>::iterator itApplier;
std::map<Key, UID>::iterator itApplier;
while (itlow != itup) {
Standalone<MutationRef> curm; // current mutation
curm.type = m.type;
@ -486,7 +603,7 @@ bool concatenateBackupMutationForLogFile(std::map<Standalone<StringRef>, Standal
std::map<Standalone<StringRef>, uint32_t>& mutationPartMap = *pMutationPartMap;
const int key_prefix_len = sizeof(uint8_t) + sizeof(Version) + sizeof(uint32_t);
BackupStringRefReader readerKey(key_input, restore_corrupted_data()); // read key_input!
StringRefReader readerKey(key_input, restore_corrupted_data()); // read key_input!
int logRangeMutationFirstLength = key_input.size() - key_prefix_len;
bool concatenated = false;
@ -508,24 +625,27 @@ bool concatenateBackupMutationForLogFile(std::map<Standalone<StringRef>, Standal
// Use commitVersion as id
Standalone<StringRef> id = StringRef((uint8_t*)&commitVersion, sizeof(Version));
if (mutationMap.find(id) == mutationMap.end()) {
auto it = mutationMap.find(id);
if (it == mutationMap.end()) {
mutationMap.insert(std::make_pair(id, val_input));
if (part != 0) {
TraceEvent(SevError, "FastRestore").detail("FirstPartNotZero", part).detail("KeyInput", getHexString(key_input));
TraceEvent(SevError, "FastRestore")
.detail("FirstPartNotZero", part)
.detail("KeyInput", getHexString(key_input));
}
mutationPartMap.insert(std::make_pair(id, part));
} else { // Concatenate the val string with the same commitVersion
mutationMap[id] =
mutationMap[id].contents().withSuffix(val_input.contents()); // Assign the new Areana to the map's value
if (part != (mutationPartMap[id] + 1)) {
it->second = it->second.contents().withSuffix(val_input.contents()); // Assign the new Areana to the map's value
auto& currentPart = mutationPartMap[id];
if (part != (currentPart + 1)) {
// Check if the same range or log file has been processed more than once!
TraceEvent(SevError, "FastRestore")
.detail("CurrentPart1", mutationPartMap[id])
.detail("CurrentPart2", part)
.detail("KeyInput", getHexString(key_input))
.detail("Hint", "Check if the same range or log file has been processed more than once");
.detail("CurrentPart1", currentPart)
.detail("CurrentPart2", part)
.detail("KeyInput", getHexString(key_input))
.detail("Hint", "Check if the same range or log file has been processed more than once");
}
mutationPartMap[id] = part;
currentPart = part;
concatenated = true;
}
@ -556,13 +676,14 @@ void _parseSerializedMutation(std::map<LoadingParam, VersionedMutationsMap>::ite
StringRef k = m.first.contents();
StringRef val = m.second.contents();
BackupStringRefReader kReader(k, restore_corrupted_data());
StringRefReader kReader(k, restore_corrupted_data());
uint64_t commitVersion = kReader.consume<uint64_t>(); // Consume little Endian data
// We have already filter the commit not in [beginVersion, endVersion) when we concatenate kv pair in log file
ASSERT_WE_THINK(asset.isInVersionRange(commitVersion));
kvOps.insert(std::make_pair(commitVersion, MutationsVec()));
auto it = kvOps.insert(std::make_pair(LogMessageVersion(commitVersion), MutationsVec()));
ASSERT(it.second); // inserted is true
BackupStringRefReader vReader(val, restore_corrupted_data());
StringRefReader vReader(val, restore_corrupted_data());
vReader.consume<uint64_t>(); // Consume the includeVersion
// TODO(xumengpanda): verify the protocol version is compatible and raise error if needed
@ -600,7 +721,7 @@ void _parseSerializedMutation(std::map<LoadingParam, VersionedMutationsMap>::ite
TraceEvent(SevFRMutationInfo, "FastRestore_VerboseDebug")
.detail("CommitVersion", commitVersion)
.detail("ParsedMutation", mutation.toString());
kvOps[commitVersion].push_back_deep(kvOps[commitVersion].arena(), mutation);
it.first->second.push_back_deep(it.first->second.arena(), mutation);
// Sampling (FASTRESTORE_SAMPLING_PERCENT%) data
if (deterministicRandom()->random01() * 100 < SERVER_KNOBS->FASTRESTORE_SAMPLING_PERCENT) {
samples.push_back_deep(samples.arena(), mutation);
@ -612,6 +733,11 @@ void _parseSerializedMutation(std::map<LoadingParam, VersionedMutationsMap>::ite
}
// Parsing the data blocks in a range file
// kvOpsIter: saves the parsed versioned-mutations for the sepcific LoadingParam;
// samplesIter: saves the sampled mutations from the parsed versioned-mutations;
// bc: backup container to read the backup file
// version: the version the parsed mutations should be at
// asset: RestoreAsset about which backup data should be parsed
ACTOR static Future<Void> _parseRangeFileToMutationsOnLoader(
std::map<LoadingParam, VersionedMutationsMap>::iterator kvOpsIter,
std::map<LoadingParam, MutationsVec>::iterator samplesIter, LoaderCounters* cc, Reference<IBackupContainer> bc,
@ -662,25 +788,25 @@ ACTOR static Future<Void> _parseRangeFileToMutationsOnLoader(
// Now data only contains the kv mutation within restoreRange
VectorRef<KeyValueRef> data = blockData.slice(rangeStart, rangeEnd);
int start = 0;
int end = data.size();
// Convert KV in data into mutations in kvOps
for (int i = start; i < end; ++i) {
// Note we give INT_MAX as the sub sequence number to override any log mutations.
const LogMessageVersion msgVersion(version, std::numeric_limits<int32_t>::max());
// Convert KV in data into SET mutations of different keys in kvOps
for (const KeyValueRef& kv : data) {
// NOTE: The KV pairs in range files are the real KV pairs in original DB.
// Should NOT add prefix or remove surfix for the backup data!
MutationRef m(MutationRef::Type::SetValue, data[i].key,
data[i].value); // ASSUME: all operation in range file is set.
MutationRef m(MutationRef::Type::SetValue, kv.key,
kv.value); // ASSUME: all operation in range file is set.
cc->loadedRangeBytes += m.totalSize();
// We cache all kv operations into kvOps, and apply all kv operations later in one place
kvOps.insert(std::make_pair(version, MutationsVec()));
auto it = kvOps.insert(std::make_pair(msgVersion, MutationsVec()));
TraceEvent(SevFRMutationInfo, "FastRestore_VerboseDebug")
.detail("CommitVersion", version)
.detail("ParsedMutationKV", m.toString());
ASSERT_WE_THINK(kvOps.find(version) != kvOps.end());
kvOps[version].push_back_deep(kvOps[version].arena(), m);
it.first->second.push_back_deep(it.first->second.arena(), m);
// Sampling (FASTRESTORE_SAMPLING_PERCENT%) data
if (deterministicRandom()->random01() * 100 < SERVER_KNOBS->FASTRESTORE_SAMPLING_PERCENT) {
cc->sampledRangeBytes += m.totalSize();
@ -691,9 +817,12 @@ ACTOR static Future<Void> _parseRangeFileToMutationsOnLoader(
return Void();
}
// Parse data blocks in a log file into a vector of <string, string> pairs. Each pair.second contains the mutations at a
// version encoded in pair.first Step 1: decodeLogFileBlock into <string, string> pairs Step 2: Concatenate the
// pair.second of pairs with the same pair.first.
// Parse data blocks in a log file into a vector of <string, string> pairs.
// Each pair.second contains the mutations at a version encoded in pair.first;
// Step 1: decodeLogFileBlock into <string, string> pairs;
// Step 2: Concatenate the second of pairs with the same pair.first.
// pProcessedFileOffset: ensure each data block is processed in order exactly once;
// pMutationMap: concatenated mutation list string at the mutation's commit version
ACTOR static Future<Void> _parseLogFileToMutationsOnLoader(NotifiedVersion* pProcessedFileOffset,
SerializedMutationListMap* pMutationMap,
SerializedMutationPartMap* pMutationPartMap,
@ -712,16 +841,9 @@ ACTOR static Future<Void> _parseLogFileToMutationsOnLoader(NotifiedVersion* pPro
wait(pProcessedFileOffset->whenAtLeast(asset.offset));
if (pProcessedFileOffset->get() == asset.offset) {
int start = 0;
int end = data.size();
int numConcatenated = 0;
for (int i = start; i < end; ++i) {
// Key k = data[i].key.withPrefix(mutationLogPrefix);
// ValueRef v = data[i].value;
for (const KeyValueRef& kv : data) {
// Concatenate the backuped param1 and param2 (KV) at the same version.
bool concatenated =
concatenateBackupMutationForLogFile(pMutationMap, pMutationPartMap, data[i].key, data[i].value, asset);
numConcatenated += (concatenated ? 1 : 0);
concatenateBackupMutationForLogFile(pMutationMap, pMutationPartMap, kv.key, kv.value, asset);
}
pProcessedFileOffset->set(asset.offset + asset.len);
}

View File

@ -36,8 +36,9 @@
#include "flow/actorcompiler.h" // This must be the last #include.
ACTOR static Future<Void> clearDB(Database cx);
ACTOR static Future<Void> collectBackupFiles(Reference<IBackupContainer> bc, std::vector<RestoreFileFR>* rangeFiles,
std::vector<RestoreFileFR>* logFiles, Database cx, RestoreRequest request);
ACTOR static Future<Version> collectBackupFiles(Reference<IBackupContainer> bc, std::vector<RestoreFileFR>* rangeFiles,
std::vector<RestoreFileFR>* logFiles, Database cx,
RestoreRequest request);
ACTOR static Future<Version> processRestoreRequest(Reference<RestoreMasterData> self, Database cx, RestoreRequest request);
ACTOR static Future<Void> startProcessRestoreRequests(Reference<RestoreMasterData> self, Database cx);
@ -276,7 +277,8 @@ ACTOR static Future<Version> processRestoreRequest(Reference<RestoreMasterData>
self->initBackupContainer(request.url);
// Get all backup files' description and save them to files
wait(collectBackupFiles(self->bc, &rangeFiles, &logFiles, cx, request));
Version targetVersion = wait(collectBackupFiles(self->bc, &rangeFiles, &logFiles, cx, request));
ASSERT(targetVersion > 0);
std::sort(rangeFiles.begin(), rangeFiles.end());
std::sort(logFiles.begin(), logFiles.end(), [](RestoreFileFR const& f1, RestoreFileFR const& f2) -> bool {
@ -284,7 +286,8 @@ ACTOR static Future<Version> processRestoreRequest(Reference<RestoreMasterData>
std::tie(f2.endVersion, f2.beginVersion, f2.fileIndex, f2.fileName);
});
self->buildVersionBatches(rangeFiles, logFiles, &self->versionBatches); // Divide files into version batches
// Divide files into version batches.
self->buildVersionBatches(rangeFiles, logFiles, &self->versionBatches, targetVersion);
self->dumpVersionBatches(self->versionBatches);
state std::vector<Future<Void>> fBatches;
@ -314,8 +317,7 @@ ACTOR static Future<Version> processRestoreRequest(Reference<RestoreMasterData>
TraceEvent("FastRestoreMasterDispatchVersionBatches")
.detail("BatchIndex", batchIndex)
.detail("BatchSize", versionBatch->size)
.detail("RunningVersionBatches", self->runningVersionBatches.get())
.detail("Start", now());
.detail("RunningVersionBatches", self->runningVersionBatches.get());
self->batch[batchIndex] = Reference<MasterBatchData>(new MasterBatchData());
self->batchStatus[batchIndex] = Reference<MasterBatchStatus>(new MasterBatchStatus());
fBatches.push_back(distributeWorkloadPerVersionBatch(self, batchIndex, cx, request, *versionBatch));
@ -335,12 +337,7 @@ ACTOR static Future<Void> loadFilesOnLoaders(Reference<MasterBatchData> batchDat
Database cx, RestoreRequest request, VersionBatch versionBatch,
bool isRangeFile) {
// set is internally sorted
std::set<RestoreFileFR>* files = nullptr;
if (isRangeFile) {
files = &versionBatch.rangeFiles;
} else {
files = &versionBatch.logFiles;
}
std::set<RestoreFileFR>* files = isRangeFile ? &versionBatch.rangeFiles : &versionBatch.logFiles;
TraceEvent("FastRestoreMasterPhaseLoadFilesStart")
.detail("BatchIndex", batchIndex)
@ -374,11 +371,14 @@ ACTOR static Future<Void> loadFilesOnLoaders(Reference<MasterBatchData> batchDat
param.asset.uid = deterministicRandom()->randomUniqueID();
param.asset.filename = file.fileName;
param.asset.fileIndex = file.fileIndex;
param.asset.partitionId = file.partitionId;
param.asset.offset = 0;
param.asset.len = file.fileSize;
param.asset.range = request.range;
param.asset.beginVersion = versionBatch.beginVersion;
param.asset.endVersion = versionBatch.endVersion;
param.asset.endVersion = (isRangeFile || request.targetVersion == -1)
? versionBatch.endVersion
: std::min(versionBatch.endVersion, request.targetVersion + 1);
TraceEvent("FastRestoreMasterPhaseLoadFiles")
.detail("BatchIndex", batchIndex)
@ -619,14 +619,12 @@ void splitKeyRangeForAppliers(Reference<MasterBatchData> batchData,
}
std::set<Key>::iterator splitter = keyrangeSplitter.begin();
int i = 0;
batchData->rangeToApplier.clear();
for (auto& applier : appliersInterf) {
if (splitter == keyrangeSplitter.end()) {
break; // Not all appliers will be used
}
batchData->rangeToApplier[*splitter] = applier.first;
i++;
splitter++;
}
ASSERT(batchData->rangeToApplier.size() > 0);
@ -677,20 +675,29 @@ ACTOR static Future<Standalone<VectorRef<RestoreRequest>>> collectRestoreRequest
}
// Collect the backup files' description into output_files by reading the backupContainer bc.
ACTOR static Future<Void> collectBackupFiles(Reference<IBackupContainer> bc, std::vector<RestoreFileFR>* rangeFiles,
std::vector<RestoreFileFR>* logFiles, Database cx,
RestoreRequest request) {
state BackupDescription desc = wait(bc->describeBackup());
// Returns the restore target version.
ACTOR static Future<Version> collectBackupFiles(Reference<IBackupContainer> bc, std::vector<RestoreFileFR>* rangeFiles,
std::vector<RestoreFileFR>* logFiles, Database cx,
RestoreRequest request) {
state BackupDescription desc = wait(bc->describePartitionedBackup());
// Convert version to real time for operators to read the BackupDescription desc.
wait(desc.resolveVersionTimes(cx));
TraceEvent("FastRestoreMasterPhaseCollectBackupFilesStart").detail("BackupDesc", desc.toString());
if (request.targetVersion == invalidVersion && desc.maxRestorableVersion.present()) {
request.targetVersion = desc.maxRestorableVersion.get();
}
Optional<RestorableFileSet> restorable = wait(bc->getRestoreSet(request.targetVersion));
TraceEvent("FastRestoreMasterPhaseCollectBackupFilesStart")
.detail("TargetVersion", request.targetVersion)
.detail("BackupDesc", desc.toString());
if (g_network->isSimulated()) {
std::cout << "Restore to version: " << request.targetVersion << "\nBackupDesc: \n" << desc.toString() << "\n\n";
}
Optional<RestorableFileSet> restorable =
wait(SERVER_KNOBS->FASTRESTORE_USE_PARTITIONED_LOGS ? bc->getPartitionedRestoreSet(request.targetVersion)
: bc->getRestoreSet(request.targetVersion));
if (!restorable.present()) {
TraceEvent(SevWarn, "FastRestoreMasterPhaseCollectBackupFiles").detail("NotRestorable", request.targetVersion);
@ -707,7 +714,7 @@ ACTOR static Future<Void> collectBackupFiles(Reference<IBackupContainer> bc, std
if (f.fileSize <= 0) {
continue;
}
RestoreFileFR file(f.version, f.fileName, true, f.blockSize, f.fileSize, f.version, f.version);
RestoreFileFR file(f);
TraceEvent("FastRestoreMasterPhaseCollectBackupFiles").detail("RangeFileFR", file.toString());
uniqueRangeFiles.insert(file);
}
@ -716,7 +723,7 @@ ACTOR static Future<Void> collectBackupFiles(Reference<IBackupContainer> bc, std
if (f.fileSize <= 0) {
continue;
}
RestoreFileFR file(f.beginVersion, f.fileName, false, f.blockSize, f.fileSize, f.endVersion, f.beginVersion);
RestoreFileFR file(f);
TraceEvent("FastRestoreMasterPhaseCollectBackupFiles").detail("LogFileFR", file.toString());
logFiles->push_back(file);
uniqueLogFiles.insert(file);
@ -729,7 +736,7 @@ ACTOR static Future<Void> collectBackupFiles(Reference<IBackupContainer> bc, std
.detail("BackupDesc", desc.toString())
.detail("RangeFiles", rangeFiles->size())
.detail("LogFiles", logFiles->size());
return Void();
return request.targetVersion;
}
ACTOR static Future<Void> clearDB(Database cx) {

View File

@ -231,19 +231,17 @@ struct RestoreMasterData : RestoreRoleData, public ReferenceCounted<RestoreMaste
}
++rangeIdx;
}
int logIdx = 0;
std::vector<RestoreFileFR> retLogs;
// Scan all logFiles every time to avoid assumption on log files' version ranges.
// For example, we do not assume each version range only exists in one log file
while (logIdx < logFiles.size()) {
Version begin = std::max(prevVersion, logFiles[logIdx].beginVersion);
Version end = std::min(nextVersion, logFiles[logIdx].endVersion);
for (const auto& file : logFiles) {
Version begin = std::max(prevVersion, file.beginVersion);
Version end = std::min(nextVersion, file.endVersion);
if (begin < end) { // logIdx file overlap in [prevVersion, nextVersion)
double ratio = (end - begin) * 1.0 / (logFiles[logIdx].endVersion - logFiles[logIdx].beginVersion);
size += logFiles[logIdx].fileSize * ratio;
retLogs.push_back(logFiles[logIdx]);
double ratio = (end - begin) * 1.0 / (file.endVersion - file.beginVersion);
size += file.fileSize * ratio;
retLogs.push_back(file);
}
++logIdx;
}
return std::make_tuple(size, rangeIdx, retLogs);
}
@ -251,13 +249,13 @@ struct RestoreMasterData : RestoreRoleData, public ReferenceCounted<RestoreMaste
// Split backup files into version batches, each of which has similar data size
// Input: sorted range files, sorted log files;
// Output: a set of version batches whose size is less than SERVER_KNOBS->FASTRESTORE_VERSIONBATCH_MAX_BYTES
// and each mutation in backup files is included in the version batches exactly once.
// and each mutation in backup files is included in the version batches exactly once.
// Assumption 1: input files has no empty files;
// Assumption 2: range files at one version <= FASTRESTORE_VERSIONBATCH_MAX_BYTES.
// Note: We do not allow a versionBatch size larger than the FASTRESTORE_VERSIONBATCH_MAX_BYTES because the range
// file size at a version depends on the number of backupAgents and its upper bound is hard to get.
void buildVersionBatches(const std::vector<RestoreFileFR>& rangeFiles, const std::vector<RestoreFileFR>& logFiles,
std::map<Version, VersionBatch>* versionBatches) {
std::map<Version, VersionBatch>* versionBatches, Version targetVersion) {
bool rewriteNextVersion = false;
int rangeIdx = 0;
int logIdx = 0; // Ensure each log file is included in version batch
@ -342,7 +340,7 @@ struct RestoreMasterData : RestoreRoleData, public ReferenceCounted<RestoreMaste
vb.logFiles.insert(log);
}
vb.endVersion = nextVersion;
vb.endVersion = std::min(nextVersion, targetVersion + 1);
prevEndVersion = vb.endVersion;
} else {
if (vb.size < 1) {
@ -378,7 +376,7 @@ struct RestoreMasterData : RestoreRoleData, public ReferenceCounted<RestoreMaste
}
// The last wip version batch has some files
if (vb.size > 0) {
vb.endVersion = nextVersion;
vb.endVersion = std::min(nextVersion, targetVersion + 1);
versionBatches->emplace(vb.beginVersion, vb);
}
}

View File

@ -51,60 +51,20 @@ struct RestoreMasterData;
struct RestoreSimpleRequest;
using VersionedMutationsMap = std::map<Version, MutationsVec>;
// Key is the (version, subsequence) of parsed backup mutations.
// Value MutationsVec is the vector of parsed backup mutations.
// For old mutation logs, the subsequence number is always 0.
// For partitioned mutation logs, each mutation has a unique LogMessageVersion.
// Note for partitioned logs, one LogMessageVersion can have multiple mutations,
// because a clear mutation may be split into several smaller clear mutations by
// backup workers.
using VersionedMutationsMap = std::map<LogMessageVersion, MutationsVec>;
ACTOR Future<Void> isSchedulable(Reference<RestoreRoleData> self, int actorBatchIndex, std::string name);
ACTOR Future<Void> handleHeartbeat(RestoreSimpleRequest req, UID id);
ACTOR Future<Void> handleInitVersionBatchRequest(RestoreVersionBatchRequest req, Reference<RestoreRoleData> self);
void handleFinishRestoreRequest(const RestoreFinishRequest& req, Reference<RestoreRoleData> self);
// Helper class for reading restore data from a buffer and throwing the right errors.
// This struct is mostly copied from StringRefReader. We add a sanity check in this struct.
// We want to decouple code between fast restore and old restore. So we keep this duplicate struct
struct BackupStringRefReader {
BackupStringRefReader(StringRef s = StringRef(), Error e = Error())
: rptr(s.begin()), end(s.end()), failure_error(e), str_size(s.size()) {}
// Return remainder of data as a StringRef
StringRef remainder() { return StringRef(rptr, end - rptr); }
// Return a pointer to len bytes at the current read position and advance read pos
// Consume a little-Endian data. Since we only run on little-Endian machine, the data on storage is little Endian
const uint8_t* consume(unsigned int len) {
if (rptr == end && len != 0) throw end_of_stream();
const uint8_t* p = rptr;
rptr += len;
if (rptr > end) {
printf("[ERROR] BackupStringRefReader throw error! string length:%d\n", str_size);
printf("!!!!!!!!!!!![ERROR]!!!!!!!!!!!!!! Worker may die due to the error. Master will stuck when a worker "
"die\n");
throw failure_error;
}
return p;
}
// Return a T from the current read position and advance read pos
template <typename T>
const T consume() {
return *(const T*)consume(sizeof(T));
}
// Functions for consuming big endian (network byte oselfer) integers.
// Consumes a big endian number, swaps it to little endian, and returns it.
int32_t consumeNetworkInt32() { return (int32_t)bigEndian32((uint32_t)consume<int32_t>()); }
uint32_t consumeNetworkUInt32() { return bigEndian32(consume<uint32_t>()); }
// Convert big Endian value (e.g., encoded in log file) into a littleEndian uint64_t value.
int64_t consumeNetworkInt64() { return (int64_t)bigEndian64((uint32_t)consume<int64_t>()); }
uint64_t consumeNetworkUInt64() { return bigEndian64(consume<uint64_t>()); }
bool eof() { return rptr == end; }
const uint8_t *rptr, *end;
const int str_size;
Error failure_error;
};
class RoleVersionBatchState {
public:
static const int INVALID = -1;

View File

@ -39,6 +39,7 @@
//#define SevFRMutationInfo SevInfo
using MutationsVec = Standalone<VectorRef<MutationRef>>;
using SubSequenceVec = Standalone<VectorRef<uint32_t>>;
enum class RestoreRole { Invalid = 0, Master = 1, Loader, Applier };
BINARY_SERIALIZABLE(RestoreRole);

View File

@ -816,7 +816,6 @@ void SimulationConfig::generateNormalConfig(int minimumReplication, int minimumR
set_config(format("log_spill:=%d", logSpill));
int logVersion = deterministicRandom()->randomInt( TLogVersion::MIN_RECRUITABLE, TLogVersion::MAX_SUPPORTED+1 );
set_config(format("log_version:=%d", logVersion));
set_config("backup_worker_enabled:=1");
} else {
if (deterministicRandom()->random01() < 0.7)
set_config(format("log_version:=%d", TLogVersion::MAX_SUPPORTED));
@ -824,6 +823,10 @@ void SimulationConfig::generateNormalConfig(int minimumReplication, int minimumR
set_config(format("log_spill:=%d", TLogSpillType::DEFAULT));
}
if (deterministicRandom()->random01() < 0.5) {
set_config("backup_worker_enabled:=1");
}
if(generateFearless || (datacenters == 2 && deterministicRandom()->random01() < 0.5)) {
//The kill region workload relies on the fact that all "0", "2", and "4" are all of the possible primary dcids.
StatusObject primaryObj;

View File

@ -41,6 +41,7 @@ ACTOR Future<Version> minVersionWhenReady(Future<Void> f, std::vector<Future<TLo
return minVersion;
}
// TagPartitionedLogSystem info in old epoch
struct OldLogData {
std::vector<Reference<LogSet>> tLogs;
int32_t logRouterTags;
@ -165,7 +166,7 @@ OldTLogCoreData::OldTLogCoreData(const OldLogData& oldData)
struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogSystem> {
const UID dbgid;
LogSystemType logSystemType;
std::vector<Reference<LogSet>> tLogs;
std::vector<Reference<LogSet>> tLogs; // LogSets in different locations: primary, remote or satellite
int expectedLogSets;
int logRouterTags;
int txsTags;
@ -191,31 +192,32 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
Optional<Version> recoverAt;
Optional<Version> recoveredAt;
Version knownCommittedVersion;
Version backupStartVersion = invalidVersion; // max(tLogs[0].startVersion, previous epochEnd).
LocalityData locality;
std::map< std::pair<UID, Tag>, std::pair<Version, Version> > outstandingPops; // For each currently running popFromLog actor, (log server #, tag)->popped version
Optional<PromiseStream<Future<Void>>> addActor;
ActorCollection popActors;
std::vector<OldLogData> oldLogData;
std::vector<OldLogData> oldLogData; // each element has the log info. in one old epoch.
AsyncTrigger logSystemConfigChanged;
TagPartitionedLogSystem(UID dbgid, LocalityData locality, LogEpoch e,
Optional<PromiseStream<Future<Void>>> addActor = Optional<PromiseStream<Future<Void>>>())
: dbgid(dbgid), logSystemType(LogSystemType::empty), expectedLogSets(0), logRouterTags(0), txsTags(0),
repopulateRegionAntiQuorum(0), epoch(e), oldestBackupEpoch(e), recoveryCompleteWrittenToCoreState(false),
repopulateRegionAntiQuorum(0), epoch(e), oldestBackupEpoch(0), recoveryCompleteWrittenToCoreState(false),
locality(locality), remoteLogsWrittenToCoreState(false), hasRemoteServers(false), stopped(false),
addActor(addActor), popActors(false) {}
void stopRejoins() override { rejoins = Future<Void>(); }
void stopRejoins() final { rejoins = Future<Void>(); }
void addref() override {
void addref() final {
ReferenceCounted<TagPartitionedLogSystem>::addref();
}
void delref() override {
void delref() final {
ReferenceCounted<TagPartitionedLogSystem>::delref();
}
std::string describe() override {
std::string describe() final {
std::string result;
for( int i = 0; i < tLogs.size(); i++ ) {
result += format("%d: ", i);
@ -226,7 +228,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
return result;
}
UID getDebugID() override { return dbgid; }
UID getDebugID() final { return dbgid; }
void addPseudoLocality(int8_t locality) {
ASSERT(locality < 0);
@ -236,7 +238,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
}
}
Tag getPseudoPopTag(Tag tag, ProcessClass::ClassType type) override {
Tag getPseudoPopTag(Tag tag, ProcessClass::ClassType type) final {
switch (type) {
case ProcessClass::LogRouterClass:
if (tag.locality == tagLocalityLogRouter) {
@ -258,10 +260,10 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
return tag;
}
bool hasPseudoLocality(int8_t locality) override { return pseudoLocalities.count(locality) > 0; }
bool hasPseudoLocality(int8_t locality) final { return pseudoLocalities.count(locality) > 0; }
// Return the min version of all pseudoLocalities, i.e., logRouter and backupTag
Version popPseudoLocalityTag(Tag tag, Version upTo) override {
Version popPseudoLocalityTag(Tag tag, Version upTo) final {
ASSERT(isPseudoLocality(tag.locality) && hasPseudoLocality(tag.locality));
Version& localityVersion = pseudoLocalityPopVersion[tag];
@ -308,6 +310,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
}
logSystem->logSystemType = lsConf.logSystemType;
logSystem->oldestBackupEpoch = lsConf.oldestBackupEpoch;
return logSystem;
}
@ -336,7 +339,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
return logSystem;
}
void toCoreState(DBCoreState& newState) override {
void toCoreState(DBCoreState& newState) final {
if( recoveryComplete.isValid() && recoveryComplete.isError() )
throw recoveryComplete.getError();
@ -358,7 +361,10 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
}
newState.oldTLogData.clear();
if(!recoveryComplete.isValid() || !recoveryComplete.isReady() || (repopulateRegionAntiQuorum == 0 && (!remoteRecoveryComplete.isValid() || !remoteRecoveryComplete.isReady()))) {
if (!recoveryComplete.isValid() || !recoveryComplete.isReady() ||
(repopulateRegionAntiQuorum == 0 &&
(!remoteRecoveryComplete.isValid() || !remoteRecoveryComplete.isReady())) ||
epoch != oldestBackupEpoch) {
for (const auto& oldData : oldLogData) {
newState.oldTLogData.emplace_back(oldData);
TraceEvent("BWToCore")
@ -371,11 +377,11 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
newState.logSystemType = logSystemType;
}
bool remoteStorageRecovered() override {
bool remoteStorageRecovered() final {
return remoteRecoveryComplete.isValid() && remoteRecoveryComplete.isReady();
}
Future<Void> onCoreStateChanged() override {
Future<Void> onCoreStateChanged() final {
std::vector<Future<Void>> changes;
changes.push_back(Never());
if(recoveryComplete.isValid() && !recoveryComplete.isReady()) {
@ -387,10 +393,11 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
if(remoteRecoveryComplete.isValid() && !remoteRecoveryComplete.isReady()) {
changes.push_back(remoteRecoveryComplete);
}
changes.push_back(backupWorkerChanged.onTrigger()); // changes to oldestBackupEpoch
return waitForAny(changes);
}
void coreStateWritten(DBCoreState const& newState) override {
void coreStateWritten(DBCoreState const& newState) final {
if (!newState.oldTLogData.size()) {
recoveryCompleteWrittenToCoreState.set(true);
}
@ -403,7 +410,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
}
}
Future<Void> onError() override {
Future<Void> onError() final {
return onError_internal(this);
}
@ -478,7 +485,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
}
Future<Version> push(Version prevVersion, Version version, Version knownCommittedVersion,
Version minKnownCommittedVersion, LogPushData& data, Optional<UID> debugID) override {
Version minKnownCommittedVersion, LogPushData& data, Optional<UID> debugID) final {
// FIXME: Randomize request order as in LegacyLogSystem?
vector<Future<Void>> quorumResults;
vector<Future<TLogCommitReply>> allReplies;
@ -650,7 +657,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
}
}
Reference<IPeekCursor> peek( UID dbgid, Version begin, Optional<Version> end, Tag tag, bool parallelGetMore ) override {
Reference<IPeekCursor> peek( UID dbgid, Version begin, Optional<Version> end, Tag tag, bool parallelGetMore ) final {
if(!tLogs.size()) {
TraceEvent("TLogPeekNoLogSets", dbgid).detail("Tag", tag.toString()).detail("Begin", begin);
return Reference<ILogSystem::ServerPeekCursor>( new ILogSystem::ServerPeekCursor( Reference<AsyncVar<OptionalInterface<TLogInterface>>>(), tag, begin, getPeekEnd(), false, false ) );
@ -663,7 +670,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
}
}
Reference<IPeekCursor> peek(UID dbgid, Version begin, Optional<Version> end, std::vector<Tag> tags, bool parallelGetMore) override {
Reference<IPeekCursor> peek(UID dbgid, Version begin, Optional<Version> end, std::vector<Tag> tags, bool parallelGetMore) final {
if(tags.empty()) {
TraceEvent("TLogPeekNoTags", dbgid).detail("Begin", begin);
return Reference<ILogSystem::ServerPeekCursor>( new ILogSystem::ServerPeekCursor( Reference<AsyncVar<OptionalInterface<TLogInterface>>>(), invalidTag, begin, getPeekEnd(), false, false ) );
@ -788,7 +795,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
}
}
Reference<IPeekCursor> peekTxs(UID dbgid, Version begin, int8_t peekLocality, Version localEnd, bool canDiscardPopped) override {
Reference<IPeekCursor> peekTxs(UID dbgid, Version begin, int8_t peekLocality, Version localEnd, bool canDiscardPopped) final {
Version end = getEnd();
if(!tLogs.size()) {
TraceEvent("TLogPeekTxsNoLogs", dbgid);
@ -870,7 +877,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
}
}
Reference<IPeekCursor> peekSingle(UID dbgid, Version begin, Tag tag, std::vector<std::pair<Version,Tag>> history) override {
Reference<IPeekCursor> peekSingle(UID dbgid, Version begin, Tag tag, std::vector<std::pair<Version,Tag>> history) final {
while(history.size() && begin >= history.back().first) {
history.pop_back();
}
@ -895,7 +902,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
}
}
Reference<IPeekCursor> peekLogRouter(UID dbgid, Version begin, Tag tag) override {
Reference<IPeekCursor> peekLogRouter(UID dbgid, Version begin, Tag tag) final {
bool found = false;
for (const auto& log : tLogs) {
found = log->hasLogRouter(dbgid) || log->hasBackupWorker(dbgid);
@ -994,7 +1001,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
return Reference<ILogSystem::ServerPeekCursor>( new ILogSystem::ServerPeekCursor( Reference<AsyncVar<OptionalInterface<TLogInterface>>>(), tag, begin, getPeekEnd(), false, false ) );
}
Version getKnownCommittedVersion() override {
Version getKnownCommittedVersion() final {
Version result = invalidVersion;
for(auto& it : lockResults) {
auto versions = TagPartitionedLogSystem::getDurableVersion(dbgid, it);
@ -1005,7 +1012,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
return result;
}
Future<Void> onKnownCommittedVersionChange() override {
Future<Void> onKnownCommittedVersionChange() final {
std::vector<Future<Void>> result;
for(auto& it : lockResults) {
result.push_back(TagPartitionedLogSystem::getDurableVersionChanged(it));
@ -1046,7 +1053,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
}
}
void popTxs(Version upTo, int8_t popLocality) override {
void popTxs(Version upTo, int8_t popLocality) final {
if( getTLogVersion() < TLogVersion::V4 ) {
pop(upTo, txsTag, 0, popLocality);
} else {
@ -1056,26 +1063,34 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
}
}
void pop(Version upTo, Tag tag, Version durableKnownCommittedVersion, int8_t popLocality) override {
void pop(Version upTo, Tag tag, Version durableKnownCommittedVersion, int8_t popLocality) final {
if (upTo <= 0) return;
if( tag.locality == tagLocalityRemoteLog) {
if (tag.locality == tagLocalityRemoteLog) {
popLogRouter(upTo, tag, durableKnownCommittedVersion, popLocality);
return;
}
for(auto& t : tLogs) {
if(t->locality == tagLocalitySpecial || t->locality == tag.locality || tag.locality == tagLocalityUpgraded || (tag.locality < 0 && ((popLocality == tagLocalityInvalid) == t->isLocal))) {
for (auto& t : tLogs) {
if (t->locality == tagLocalitySpecial || t->locality == tag.locality ||
tag.locality == tagLocalityUpgraded ||
(tag.locality < 0 && ((popLocality == tagLocalityInvalid) == t->isLocal))) {
for(auto& log : t->logServers) {
Version prev = outstandingPops[std::make_pair(log->get().id(),tag)].first;
if (prev < upTo)
if (prev < upTo) {
// update pop version for popFromLog actor
outstandingPops[std::make_pair(log->get().id(),tag)] = std::make_pair(upTo, durableKnownCommittedVersion);
if (prev == 0)
}
if (prev == 0) {
// pop tag from log upto version defined in outstandingPops[].first
popActors.add( popFromLog( this, log, tag, 1.0 ) ); //< FIXME: knob
}
}
}
}
}
ACTOR static Future<Void> popFromLog( TagPartitionedLogSystem* self, Reference<AsyncVar<OptionalInterface<TLogInterface>>> log, Tag tag, double time ) {
ACTOR static Future<Void> popFromLog(TagPartitionedLogSystem* self,
Reference<AsyncVar<OptionalInterface<TLogInterface>>> log, Tag tag,
double time) {
state Version last = 0;
loop {
wait( delay(time, TaskPriority::TLogPop) );
@ -1156,7 +1171,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
return maxPopped;
}
Future<Version> getTxsPoppedVersion() override {
Future<Version> getTxsPoppedVersion() final {
return getPoppedTxs(this);
}
@ -1208,7 +1223,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
}
// Returns success after confirming that pushes in the current epoch are still possible
Future<Void> confirmEpochLive(Optional<UID> debugID) override {
Future<Void> confirmEpochLive(Optional<UID> debugID) final {
vector<Future<Void>> quorumResults;
for(auto& it : tLogs) {
if(it->isLocal && it->logServers.size()) {
@ -1219,7 +1234,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
return waitForAll(quorumResults);
}
Future<Void> endEpoch() override {
Future<Void> endEpoch() final {
std::vector<Future<Void>> lockResults;
for( auto& logSet : tLogs ) {
for( auto& log : logSet->logServers ) {
@ -1236,11 +1251,11 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
DatabaseConfiguration const& config, LogEpoch recoveryCount,
int8_t primaryLocality, int8_t remoteLocality,
std::vector<Tag> const& allTags,
Reference<AsyncVar<bool>> const& recruitmentStalled) override {
Reference<AsyncVar<bool>> const& recruitmentStalled) final {
return newEpoch( Reference<TagPartitionedLogSystem>::addRef(this), recr, fRemoteWorkers, config, recoveryCount, primaryLocality, remoteLocality, allTags, recruitmentStalled );
}
LogSystemConfig getLogSystemConfig() override {
LogSystemConfig getLogSystemConfig() final {
LogSystemConfig logSystemConfig(epoch);
logSystemConfig.logSystemType = logSystemType;
logSystemConfig.expectedLogSets = expectedLogSets;
@ -1265,7 +1280,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
return logSystemConfig;
}
Standalone<StringRef> getLogsValue() override {
Standalone<StringRef> getLogsValue() final {
vector<std::pair<UID, NetworkAddress>> logs;
vector<std::pair<UID, NetworkAddress>> oldLogs;
for(auto& t : tLogs) {
@ -1287,7 +1302,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
return logsValue( logs, oldLogs );
}
Future<Void> onLogSystemConfigChange() override {
Future<Void> onLogSystemConfigChange() final {
std::vector<Future<Void>> changes;
changes.push_back(logSystemConfigChanged.onTrigger());
for(auto& t : tLogs) {
@ -1310,7 +1325,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
return waitForAny(changes);
}
Version getEnd() override {
Version getEnd() final {
ASSERT( recoverAt.present() );
return recoverAt.get() + 1;
}
@ -1322,7 +1337,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
return std::numeric_limits<Version>::max();
}
void getPushLocations(VectorRef<Tag> tags, std::vector<int>& locations, bool allLocations) override {
void getPushLocations(VectorRef<Tag> tags, std::vector<int>& locations, bool allLocations) final {
int locationOffset = 0;
for(auto& log : tLogs) {
if(log->isLocal && log->logServers.size()) {
@ -1332,30 +1347,30 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
}
}
bool hasRemoteLogs() const override {
bool hasRemoteLogs() const final {
return logRouterTags > 0 || pseudoLocalities.size() > 0;
}
Tag getRandomRouterTag() const override {
Tag getRandomRouterTag() const final {
return Tag(tagLocalityLogRouter, deterministicRandom()->randomInt(0, logRouterTags));
}
Tag getRandomTxsTag() const override {
Tag getRandomTxsTag() const final {
return Tag(tagLocalityTxs, deterministicRandom()->randomInt(0, txsTags));
}
TLogVersion getTLogVersion() const override {
TLogVersion getTLogVersion() const final {
return tLogs[0]->tLogVersion;
}
int getLogRouterTags() const override { return logRouterTags; }
int getLogRouterTags() const final { return logRouterTags; }
Version getStartVersion() const override {
Version getBackupStartVersion() const final {
ASSERT(tLogs.size() > 0);
return tLogs[0]->startVersion;
return backupStartVersion;
}
std::map<LogEpoch, ILogSystem::EpochTagsVersionsInfo> getOldEpochTagsVersionsInfo() const override {
std::map<LogEpoch, ILogSystem::EpochTagsVersionsInfo> getOldEpochTagsVersionsInfo() const final {
std::map<LogEpoch, EpochTagsVersionsInfo> epochInfos;
for (const auto& old : oldLogData) {
epochInfos.insert(
@ -1376,7 +1391,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
return Reference<LogSet>(nullptr);
}
void setBackupWorkers(const std::vector<InitializeBackupReply>& replies) override {
void setBackupWorkers(const std::vector<InitializeBackupReply>& replies) final {
ASSERT(tLogs.size() > 0);
Reference<LogSet> logset = tLogs[0]; // Master recruits this epoch's worker first.
@ -1393,10 +1408,11 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
}
logset->backupWorkers.push_back(worker);
}
TraceEvent("SetOldestBackupEpoch", dbgid).detail("Epoch", oldestBackupEpoch);
backupWorkerChanged.trigger();
}
bool removeBackupWorker(const BackupWorkerDoneRequest& req) override {
bool removeBackupWorker(const BackupWorkerDoneRequest& req) final {
bool removed = false;
Reference<LogSet> logset = getEpochLogSet(req.backupEpoch);
if (logset.isValid()) {
@ -1427,7 +1443,12 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
return removed;
}
LogEpoch getOldestBackupEpoch() const override { return oldestBackupEpoch; }
LogEpoch getOldestBackupEpoch() const final { return oldestBackupEpoch; }
void setOldestBackupEpoch(LogEpoch epoch) final {
oldestBackupEpoch = epoch;
backupWorkerChanged.trigger();
}
ACTOR static Future<Void> monitorLog(Reference<AsyncVar<OptionalInterface<TLogInterface>>> logServer, Reference<AsyncVar<bool>> failed) {
state Future<Void> waitFailure;
@ -2214,6 +2235,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
logSystem->oldLogData.insert(logSystem->oldLogData.end(), oldLogSystem->oldLogData.begin(), oldLogSystem->oldLogData.end());
logSystem->tLogs[0]->startVersion = oldLogSystem->knownCommittedVersion + 1;
logSystem->backupStartVersion = oldLogSystem->knownCommittedVersion + 1;
state int lockNum = 0;
while(lockNum < oldLogSystem->lockResults.size()) {
if(oldLogSystem->lockResults[lockNum].logSet->locality == primaryLocality) {
@ -2360,7 +2382,7 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
}
}
}
for( int i = 0; i < recr.satelliteTLogs.size(); i++ ) {
InitializeTLogRequest &req = sreqs[i];
req.recruitmentID = logSystem->recruitmentID;

View File

@ -170,6 +170,7 @@ struct InitializeBackupRequest {
LogEpoch backupEpoch; // The epoch the worker should work on. If different from the recruitedEpoch, then it refers
// to some previous epoch with unfinished work.
Tag routerTag;
int totalTags;
Version startVersion;
Optional<Version> endVersion;
ReplyPromise<struct InitializeBackupReply> reply;
@ -179,7 +180,7 @@ struct InitializeBackupRequest {
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, reqId, recruitedEpoch, backupEpoch, routerTag, startVersion, endVersion, reply);
serializer(ar, reqId, recruitedEpoch, backupEpoch, routerTag, totalTags, startVersion, endVersion, reply);
}
};

View File

@ -478,13 +478,20 @@ ACTOR Future<Void> updateRegistration( Reference<MasterData> self, Reference<ILo
trigger = self->registrationTrigger.onTrigger();
TraceEvent("MasterUpdateRegistration", self->dbgid).detail("RecoveryCount", self->cstate.myDBState.recoveryCount).detail("Logs", describe(logSystem->getLogSystemConfig().tLogs));
auto logSystemConfig = logSystem->getLogSystemConfig();
TraceEvent("MasterUpdateRegistration", self->dbgid)
.detail("RecoveryCount", self->cstate.myDBState.recoveryCount)
.detail("OldestBackupEpoch", logSystemConfig.oldestBackupEpoch)
.detail("Logs", describe(logSystemConfig.tLogs));
if (!self->cstateUpdated.isSet()) {
wait(sendMasterRegistration(self.getPtr(), logSystem->getLogSystemConfig(), self->provisionalProxies, self->resolvers, self->cstate.myDBState.recoveryCount, self->cstate.prevDBState.getPriorCommittedLogServers() ));
wait(sendMasterRegistration(self.getPtr(), logSystemConfig, self->provisionalProxies, self->resolvers,
self->cstate.myDBState.recoveryCount,
self->cstate.prevDBState.getPriorCommittedLogServers()));
} else {
updateLogsKey = updateLogsValue(self, cx);
wait( sendMasterRegistration( self.getPtr(), logSystem->getLogSystemConfig(), self->proxies, self->resolvers, self->cstate.myDBState.recoveryCount, vector<UID>() ) );
wait(sendMasterRegistration(self.getPtr(), logSystemConfig, self->proxies, self->resolvers,
self->cstate.myDBState.recoveryCount, vector<UID>()));
}
}
}
@ -1238,9 +1245,41 @@ ACTOR Future<Void> configurationMonitor(Reference<MasterData> self, Database cx)
}
}
ACTOR static Future<Optional<Version>> getMinBackupVersion(Reference<MasterData> self, Database cx) {
loop {
state ReadYourWritesTransaction tr(cx);
try {
tr.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr.setOption(FDBTransactionOptions::LOCK_AWARE);
Optional<Value> value = wait(tr.get(backupStartedKey));
Optional<Version> minVersion;
if (value.present()) {
auto uidVersions = decodeBackupStartedValue(value.get());
TraceEvent e("GotBackupStartKey", self->dbgid);
int i = 1;
for (auto [uid, version] : uidVersions) {
e.detail(format("BackupID%d", i), uid).detail(format("Version%d", i), version);
i++;
minVersion = minVersion.present() ? std::min(version, minVersion.get()) : version;
}
} else {
TraceEvent("EmptyBackupStartKey", self->dbgid);
}
return minVersion;
} catch (Error& e) {
wait(tr.onError(e));
}
}
}
ACTOR static Future<Void> recruitBackupWorkers(Reference<MasterData> self, Database cx) {
ASSERT(self->backupWorkers.size() > 0);
// Avoid race between a backup worker's save progress and the reads below.
wait(delay(SERVER_KNOBS->SECONDS_BEFORE_RECRUIT_BACKUP_WORKER));
state LogEpoch epoch = self->cstate.myDBState.recoveryCount;
state Reference<BackupProgress> backupProgress(
new BackupProgress(self->dbgid, self->logSystem->getOldEpochTagsVersionsInfo()));
@ -1253,7 +1292,7 @@ ACTOR static Future<Void> recruitBackupWorkers(Reference<MasterData> self, Datab
idsTags.emplace_back(deterministicRandom()->randomUniqueID(), Tag(tagLocalityLogRouter, i));
}
const Version startVersion = self->logSystem->getStartVersion();
const Version startVersion = self->logSystem->getBackupStartVersion();
state int i = 0;
for (; i < logRouterTags; i++) {
const auto& worker = self->backupWorkers[i % self->backupWorkers.size()];
@ -1261,9 +1300,10 @@ ACTOR static Future<Void> recruitBackupWorkers(Reference<MasterData> self, Datab
req.recruitedEpoch = epoch;
req.backupEpoch = epoch;
req.routerTag = idsTags[i].second;
req.totalTags = logRouterTags;
req.startVersion = startVersion;
TraceEvent("BackupRecruitment", self->dbgid)
.detail("BKID", req.reqId)
.detail("RequestID", req.reqId)
.detail("Tag", req.routerTag.toString())
.detail("Epoch", epoch)
.detail("BackupEpoch", epoch)
@ -1274,20 +1314,34 @@ ACTOR static Future<Void> recruitBackupWorkers(Reference<MasterData> self, Datab
master_backup_worker_failed()));
}
wait(gotProgress);
std::map<std::pair<LogEpoch, Version>, std::map<Tag, Version>> toRecruit = backupProgress->getUnfinishedBackup();
for (const auto& [epochVersion, tagVersions] : toRecruit) {
state Future<Optional<Version>> fMinVersion = getMinBackupVersion(self, cx);
wait(gotProgress && success(fMinVersion));
TraceEvent("MinBackupVersion", self->dbgid).detail("Version", fMinVersion.get().present() ? fMinVersion.get() : -1);
std::map<std::tuple<LogEpoch, Version, int>, std::map<Tag, Version>> toRecruit =
backupProgress->getUnfinishedBackup();
for (const auto& [epochVersionTags, tagVersions] : toRecruit) {
const Version oldEpochEnd = std::get<1>(epochVersionTags);
if (!fMinVersion.get().present() || fMinVersion.get().get() + 1 >= oldEpochEnd) {
TraceEvent("SkipBackupRecruitment", self->dbgid)
.detail("MinVersion", fMinVersion.get().present() ? fMinVersion.get() : -1)
.detail("Epoch", epoch)
.detail("OldEpoch", std::get<0>(epochVersionTags))
.detail("OldEpochEnd", oldEpochEnd);
continue;
}
for (const auto& [tag, version] : tagVersions) {
const auto& worker = self->backupWorkers[i % self->backupWorkers.size()];
i++;
InitializeBackupRequest req(deterministicRandom()->randomUniqueID());
req.recruitedEpoch = epoch;
req.backupEpoch = epochVersion.first;
req.backupEpoch = std::get<0>(epochVersionTags);
req.routerTag = tag;
req.totalTags = std::get<2>(epochVersionTags);
req.startVersion = version; // savedVersion + 1
req.endVersion = epochVersion.second - 1;
req.endVersion = std::get<1>(epochVersionTags) - 1;
TraceEvent("BackupRecruitment", self->dbgid)
.detail("BKID", req.reqId)
.detail("RequestID", req.reqId)
.detail("Tag", req.routerTag.toString())
.detail("Epoch", epoch)
.detail("BackupEpoch", req.backupEpoch)
@ -1548,6 +1602,8 @@ ACTOR Future<Void> masterCore( Reference<MasterData> self ) {
self->addActor.send(configurationMonitor(self, cx));
if (self->configuration.backupWorkerEnabled) {
self->addActor.send(recruitBackupWorkers(self, cx));
} else {
self->logSystem->setOldestBackupEpoch(self->cstate.myDBState.recoveryCount);
}
wait( Future<Void>(Never()) );

View File

@ -906,6 +906,7 @@ ACTOR Future<Void> workerServer(
// here is no, so that when running with log_version==3, all files should say V=3.
state std::map<SharedLogsKey, SharedLogsValue> sharedLogs;
state Reference<AsyncVar<UID>> activeSharedTLog(new AsyncVar<UID>());
state WorkerCache<InitializeBackupReply> backupWorkerCache;
state std::string coordFolder = abspath(_coordFolder);
@ -1164,17 +1165,24 @@ ACTOR Future<Void> workerServer(
req.reply.send(recruited);
}
when (InitializeBackupRequest req = waitNext(interf.backup.getFuture())) {
BackupInterface recruited(locality);
recruited.initEndpoints();
if (!backupWorkerCache.exists(req.reqId)) {
BackupInterface recruited(locality);
recruited.initEndpoints();
startRole(Role::BACKUP, recruited.id(), interf.id());
DUMPTOKEN(recruited.waitFailure);
startRole(Role::BACKUP, recruited.id(), interf.id());
DUMPTOKEN(recruited.waitFailure);
Future<Void> backupProcess = backupWorker(recruited, req, dbInfo);
errorForwarders.add(forwardError(errors, Role::BACKUP, recruited.id(), backupProcess));
TraceEvent("BackupInitRequest", req.reqId).detail("BackupId", recruited.id());
InitializeBackupReply reply(recruited, req.backupEpoch);
req.reply.send(reply);
ReplyPromise<InitializeBackupReply> backupReady = req.reply;
backupWorkerCache.set(req.reqId, backupReady.getFuture());
Future<Void> backupProcess = backupWorker(recruited, req, dbInfo);
backupProcess = storageCache.removeOnReady(req.reqId, backupProcess);
errorForwarders.add(forwardError(errors, Role::BACKUP, recruited.id(), backupProcess));
TraceEvent("BackupInitRequest", req.reqId).detail("BackupId", recruited.id());
InitializeBackupReply reply(recruited, req.backupEpoch);
backupReady.send(reply);
} else {
forwardPromise(req.reply, backupWorkerCache.get(req.reqId));
}
}
when( InitializeTLogRequest req = waitNext(interf.tLog.getFuture()) ) {
// For now, there's a one-to-one mapping of spill type to TLogVersion.

View File

@ -180,7 +180,8 @@ struct BackupAndParallelRestoreCorrectnessWorkload : TestWorkload {
try {
wait(backupAgent->submitBackup(cx, StringRef(backupContainer), deterministicRandom()->randomInt(0, 100),
tag.toString(), backupRanges, stopDifferentialDelay ? false : true));
tag.toString(), backupRanges, stopDifferentialDelay ? false : true,
/*partitionedLog=*/true));
} catch (Error& e) {
TraceEvent("BARW_DoBackupSubmitBackupException", randomID).error(e).detail("Tag", printable(tag));
if (e.code() != error_code_backup_unneeded && e.code() != error_code_backup_duplicate) throw;
@ -209,7 +210,7 @@ struct BackupAndParallelRestoreCorrectnessWorkload : TestWorkload {
state bool restorable = false;
if(lastBackupContainer) {
state Future<BackupDescription> fdesc = lastBackupContainer->describeBackup();
state Future<BackupDescription> fdesc = lastBackupContainer->describePartitionedBackup();
wait(ready(fdesc));
if(!fdesc.isError()) {
@ -395,9 +396,9 @@ struct BackupAndParallelRestoreCorrectnessWorkload : TestWorkload {
if (!self->locked && BUGGIFY) {
TraceEvent("BARW_SubmitBackup2", randomID).detail("Tag", printable(self->backupTag));
try {
extraBackup = backupAgent.submitBackup(cx, LiteralStringRef("file://simfdb/backups/"),
deterministicRandom()->randomInt(0, 100),
self->backupTag.toString(), self->backupRanges, true);
extraBackup = backupAgent.submitBackup(
cx, LiteralStringRef("file://simfdb/backups/"), deterministicRandom()->randomInt(0, 100),
self->backupTag.toString(), self->backupRanges, true, /*partitionedLog=*/true);
} catch (Error& e) {
TraceEvent("BARW_SubmitBackup2Exception", randomID)
.error(e)
@ -430,7 +431,7 @@ struct BackupAndParallelRestoreCorrectnessWorkload : TestWorkload {
.detail("BackupTag", printable(self->backupTag));
auto container = IBackupContainer::openContainer(lastBackupContainer->getURL());
BackupDescription desc = wait(container->describeBackup());
BackupDescription desc = wait(container->describePartitionedBackup());
state Version targetVersion = -1;
if (desc.maxRestorableVersion.present()) {
@ -439,6 +440,11 @@ struct BackupAndParallelRestoreCorrectnessWorkload : TestWorkload {
} else if (deterministicRandom()->random01() < 0.1) {
targetVersion = desc.maxRestorableVersion.get();
} else if (deterministicRandom()->random01() < 0.5) {
ASSERT_WE_THINK(desc.minRestorableVersion.get() <= desc.contiguousLogEnd.get());
// This assertion can fail when contiguousLogEnd < maxRestorableVersion and
// the snapshot version > contiguousLogEnd. I.e., there is a gap between
// contiguousLogEnd and snapshot version.
// ASSERT_WE_THINK(desc.contiguousLogEnd.get() > desc.maxRestorableVersion.get());
targetVersion = deterministicRandom()->randomInt64(desc.minRestorableVersion.get(),
desc.contiguousLogEnd.get());
}