foundationdb/fdbclient/BackupContainer.actor.cpp
Andrew Noyes ff7d306b09 Merge branch 'release-6.3' into anoyes/merge-6.3-to-master
Include conflict markers for now. Will resolve.
2021-01-15 18:04:09 +00:00

1528 lines
56 KiB
C++

/*
* BackupContainer.actor.cpp
*
* This source file is part of the FoundationDB open source project
*
* Copyright 2013-2018 Apple Inc. and the FoundationDB project authors
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#include <cstdlib>
#include <ostream>
// FIXME: Trim this down
#include "flow/Platform.actor.h"
#include "fdbclient/AsyncTaskThread.h"
#include "fdbclient/BackupContainer.h"
#include "fdbclient/BackupAgent.actor.h"
#include "fdbclient/FDBTypes.h"
#include "fdbclient/JsonBuilder.h"
#include "flow/Arena.h"
#include "flow/Trace.h"
#include "flow/UnitTest.h"
#include "flow/Hash3.h"
#include "fdbrpc/AsyncFileReadAhead.actor.h"
#include "fdbrpc/simulator.h"
#include "flow/Platform.h"
#include "fdbclient/AsyncFileS3BlobStore.actor.h"
#include "fdbclient/BackupContainerAzureBlobStore.h"
#include "fdbclient/BackupContainerFileSystem.h"
#include "fdbclient/BackupContainerLocalDirectory.h"
#include "fdbclient/BackupContainerS3BlobStore.h"
#include "fdbclient/Status.h"
#include "fdbclient/SystemData.h"
#include "fdbclient/ReadYourWrites.h"
#include "fdbclient/KeyBackedTypes.h"
#include "fdbclient/RunTransaction.actor.h"
#include <algorithm>
#include <cinttypes>
#include <time.h>
#include "flow/actorcompiler.h" // has to be last include
namespace IBackupFile_impl {
ACTOR Future<Void> appendStringRefWithLen(Reference<IBackupFile> file, Standalone<StringRef> s) {
state uint32_t lenBuf = bigEndian32((uint32_t)s.size());
wait(file->append(&lenBuf, sizeof(lenBuf)));
wait(file->append(s.begin(), s.size()));
return Void();
}
} // namespace IBackupFile_impl
Future<Void> IBackupFile::appendStringRefWithLen(Standalone<StringRef> s) {
return IBackupFile_impl::appendStringRefWithLen(Reference<IBackupFile>::addRef(this), s);
}
std::string IBackupContainer::ExpireProgress::toString() const {
std::string s = step + "...";
if(total > 0) {
s += format("%d/%d (%.2f%%)", done, total, double(done) / total * 100);
}
return s;
}
void BackupFileList::toStream(FILE *fout) const {
for(const RangeFile &f : ranges) {
fprintf(fout, "range %" PRId64 " %s\n", f.fileSize, f.fileName.c_str());
}
for(const LogFile &f : logs) {
fprintf(fout, "log %" PRId64 " %s\n", f.fileSize, f.fileName.c_str());
}
for(const KeyspaceSnapshotFile &f : snapshots) {
fprintf(fout, "snapshotManifest %" PRId64 " %s\n", f.totalSize, f.fileName.c_str());
}
}
Future<Void> fetchTimes(Reference<ReadYourWritesTransaction> tr, std::map<Version, int64_t> *pVersionTimeMap) {
std::vector<Future<Void>> futures;
// Resolve each version in the map,
for(auto &p : *pVersionTimeMap) {
futures.push_back(map(timeKeeperEpochsFromVersion(p.first, tr), [=](Optional<int64_t> t) {
if(t.present())
pVersionTimeMap->at(p.first) = t.get();
else
pVersionTimeMap->erase(p.first);
return Void();
}));
}
return waitForAll(futures);
}
Future<Void> BackupDescription::resolveVersionTimes(Database cx) {
// Populate map with versions needed
versionTimeMap.clear();
for(const KeyspaceSnapshotFile &m : snapshots) {
versionTimeMap[m.beginVersion];
versionTimeMap[m.endVersion];
}
if(minLogBegin.present())
versionTimeMap[minLogBegin.get()];
if(maxLogEnd.present())
versionTimeMap[maxLogEnd.get()];
if(contiguousLogEnd.present())
versionTimeMap[contiguousLogEnd.get()];
if(minRestorableVersion.present())
versionTimeMap[minRestorableVersion.get()];
if(maxRestorableVersion.present())
versionTimeMap[maxRestorableVersion.get()];
return runRYWTransaction(cx, [=](Reference<ReadYourWritesTransaction> tr) { return fetchTimes(tr, &versionTimeMap); });
};
std::string BackupDescription::toString() const {
std::string info;
info.append(format("URL: %s\n", url.c_str()));
info.append(format("Restorable: %s\n", maxRestorableVersion.present() ? "true" : "false"));
info.append(format("Partitioned logs: %s\n", partitioned ? "true" : "false"));
auto formatVersion = [&](Version v) {
std::string s;
if(!versionTimeMap.empty()) {
auto i = versionTimeMap.find(v);
if(i != versionTimeMap.end())
s = format("%lld (%s)", v, BackupAgentBase::formatTime(i->second).c_str());
else
s = format("%lld (unknown)", v);
}
else if(maxLogEnd.present()) {
double days = double(maxLogEnd.get() - v) / (CLIENT_KNOBS->CORE_VERSIONSPERSECOND * 24 * 60 * 60);
s = format("%lld (maxLogEnd %s%.2f days)", v, days < 0 ? "+" : "-", days);
}
else {
s = format("%lld", v);
}
return s;
};
for(const KeyspaceSnapshotFile &m : snapshots) {
info.append(format("Snapshot: startVersion=%s endVersion=%s totalBytes=%lld restorable=%s expiredPct=%.2f\n",
formatVersion(m.beginVersion).c_str(), formatVersion(m.endVersion).c_str(), m.totalSize, m.restorable.orDefault(false) ? "true" : "false", m.expiredPct(expiredEndVersion)));
}
info.append(format("SnapshotBytes: %lld\n", snapshotBytes));
if(expiredEndVersion.present())
info.append(format("ExpiredEndVersion: %s\n", formatVersion(expiredEndVersion.get()).c_str()));
if(unreliableEndVersion.present())
info.append(format("UnreliableEndVersion: %s\n", formatVersion(unreliableEndVersion.get()).c_str()));
if(minLogBegin.present())
info.append(format("MinLogBeginVersion: %s\n", formatVersion(minLogBegin.get()).c_str()));
if(contiguousLogEnd.present())
info.append(format("ContiguousLogEndVersion: %s\n", formatVersion(contiguousLogEnd.get()).c_str()));
if(maxLogEnd.present())
info.append(format("MaxLogEndVersion: %s\n", formatVersion(maxLogEnd.get()).c_str()));
if(minRestorableVersion.present())
info.append(format("MinRestorableVersion: %s\n", formatVersion(minRestorableVersion.get()).c_str()));
if(maxRestorableVersion.present())
info.append(format("MaxRestorableVersion: %s\n", formatVersion(maxRestorableVersion.get()).c_str()));
if(!extendedDetail.empty())
info.append("ExtendedDetail: ").append(extendedDetail);
return info;
}
std::string BackupDescription::toJSON() const {
JsonBuilderObject doc;
doc.setKey("SchemaVersion", "1.0.0");
doc.setKey("URL", url.c_str());
doc.setKey("Restorable", maxRestorableVersion.present());
doc.setKey("Partitioned", partitioned);
auto formatVersion = [&](Version v) {
JsonBuilderObject doc;
doc.setKey("Version", v);
if(!versionTimeMap.empty()) {
auto i = versionTimeMap.find(v);
if(i != versionTimeMap.end()) {
doc.setKey("Timestamp", BackupAgentBase::formatTime(i->second));
doc.setKey("EpochSeconds", i->second);
}
}
else if(maxLogEnd.present()) {
double days = double(v - maxLogEnd.get()) / (CLIENT_KNOBS->CORE_VERSIONSPERSECOND * 24 * 60 * 60);
doc.setKey("RelativeDays", days);
}
return doc;
};
JsonBuilderArray snapshotsArray;
for(const KeyspaceSnapshotFile &m : snapshots) {
JsonBuilderObject snapshotDoc;
snapshotDoc.setKey("Start", formatVersion(m.beginVersion));
snapshotDoc.setKey("End", formatVersion(m.endVersion));
snapshotDoc.setKey("Restorable", m.restorable.orDefault(false));
snapshotDoc.setKey("TotalBytes", m.totalSize);
snapshotDoc.setKey("PercentageExpired", m.expiredPct(expiredEndVersion));
snapshotsArray.push_back(snapshotDoc);
}
doc.setKey("Snapshots", snapshotsArray);
doc.setKey("TotalSnapshotBytes", snapshotBytes);
if(expiredEndVersion.present())
doc.setKey("ExpiredEnd", formatVersion(expiredEndVersion.get()));
if(unreliableEndVersion.present())
doc.setKey("UnreliableEnd", formatVersion(unreliableEndVersion.get()));
if(minLogBegin.present())
doc.setKey("MinLogBegin", formatVersion(minLogBegin.get()));
if(contiguousLogEnd.present())
doc.setKey("ContiguousLogEnd", formatVersion(contiguousLogEnd.get()));
if(maxLogEnd.present())
doc.setKey("MaxLogEnd", formatVersion(maxLogEnd.get()));
if(minRestorableVersion.present())
doc.setKey("MinRestorablePoint", formatVersion(minRestorableVersion.get()));
if(maxRestorableVersion.present())
doc.setKey("MaxRestorablePoint", formatVersion(maxRestorableVersion.get()));
if(!extendedDetail.empty())
doc.setKey("ExtendedDetail", extendedDetail);
return doc.getJson();
}
std::string IBackupContainer::lastOpenError;
std::vector<std::string> IBackupContainer::getURLFormats() {
return {
#ifdef BUILD_AZURE_BACKUP
BackupContainerAzureBlobStore::getURLFormat(),
#endif
BackupContainerLocalDirectory::getURLFormat(),
BackupContainerS3BlobStore::getURLFormat(),
};
}
// Get an IBackupContainer based on a container URL string
Reference<IBackupContainer> IBackupContainer::openContainer(const std::string& url) {
static std::map<std::string, Reference<IBackupContainer>> m_cache;
Reference<IBackupContainer>& r = m_cache[url];
if (r) return r;
try {
StringRef u(url);
if (u.startsWith(LiteralStringRef("file://"))) {
r = Reference<IBackupContainer>(new BackupContainerLocalDirectory(url));
} else if (u.startsWith(LiteralStringRef("blobstore://"))) {
std::string resource;
// The URL parameters contain blobstore endpoint tunables as well as possible backup-specific options.
S3BlobStoreEndpoint::ParametersT backupParams;
Reference<S3BlobStoreEndpoint> bstore =
S3BlobStoreEndpoint::fromString(url, &resource, &lastOpenError, &backupParams);
if (resource.empty()) throw backup_invalid_url();
for (auto c : resource)
if (!isalnum(c) && c != '_' && c != '-' && c != '.' && c != '/') throw backup_invalid_url();
r = Reference<IBackupContainer>(new BackupContainerS3BlobStore(bstore, resource, backupParams));
}
#ifdef BUILD_AZURE_BACKUP
else if (u.startsWith(LiteralStringRef("azure://"))) {
u.eat(LiteralStringRef("azure://"));
auto address = NetworkAddress::parse(u.eat(LiteralStringRef("/")).toString());
auto containerName = u.eat(LiteralStringRef("/")).toString();
auto accountName = u.eat(LiteralStringRef("/")).toString();
r = Reference<IBackupContainer>(new BackupContainerAzureBlobStore(address, containerName, accountName));
}
#endif
else {
lastOpenError = "invalid URL prefix";
throw backup_invalid_url();
}
r->URL = url;
return r;
} catch (Error& e) {
if (e.code() == error_code_actor_cancelled) throw;
TraceEvent m(SevWarn, "BackupContainer");
m.detail("Description", "Invalid container specification. See help.");
m.detail("URL", url);
m.error(e);
if (e.code() == error_code_backup_invalid_url) m.detail("LastOpenError", lastOpenError);
throw;
}
}
// Get a list of URLS to backup containers based on some a shorter URL. This function knows about some set of supported
// URL types which support this sort of backup discovery.
ACTOR Future<std::vector<std::string>> listContainers_impl(std::string baseURL) {
try {
StringRef u(baseURL);
if (u.startsWith(LiteralStringRef("file://"))) {
std::vector<std::string> results = wait(BackupContainerLocalDirectory::listURLs(baseURL));
return results;
} else if (u.startsWith(LiteralStringRef("blobstore://"))) {
std::string resource;
S3BlobStoreEndpoint::ParametersT backupParams;
Reference<S3BlobStoreEndpoint> bstore =
S3BlobStoreEndpoint::fromString(baseURL, &resource, &IBackupContainer::lastOpenError, &backupParams);
if (!resource.empty()) {
TraceEvent(SevWarn, "BackupContainer")
.detail("Description", "Invalid backup container base URL, resource aka path should be blank.")
.detail("URL", baseURL);
throw backup_invalid_url();
}
// Create a dummy container to parse the backup-specific parameters from the URL and get a final bucket name
BackupContainerS3BlobStore dummy(bstore, "dummy", backupParams);
std::vector<std::string> results = wait(BackupContainerS3BlobStore::listURLs(bstore, dummy.getBucket()));
return results;
}
// TODO: Enable this when Azure backups are ready
/*
else if (u.startsWith(LiteralStringRef("azure://"))) {
std::vector<std::string> results = wait(BackupContainerAzureBlobStore::listURLs(baseURL));
return results;
}
*/
else {
IBackupContainer::lastOpenError = "invalid URL prefix";
throw backup_invalid_url();
}
} catch (Error& e) {
if (e.code() == error_code_actor_cancelled) throw;
TraceEvent m(SevWarn, "BackupContainer");
m.detail("Description", "Invalid backup container URL prefix. See help.");
m.detail("URL", baseURL);
m.error(e);
if (e.code() == error_code_backup_invalid_url) m.detail("LastOpenError", IBackupContainer::lastOpenError);
throw;
}
}
Future<std::vector<std::string>> IBackupContainer::listContainers(const std::string& baseURL) {
return listContainers_impl(baseURL);
}
ACTOR Future<Version> timeKeeperVersionFromDatetime(std::string datetime, Database db) {
state KeyBackedMap<int64_t, Version> versionMap(timeKeeperPrefixRange.begin);
state Reference<ReadYourWritesTransaction> tr = makeReference<ReadYourWritesTransaction>(db);
state int64_t time = BackupAgentBase::parseTime(datetime);
if (time < 0) {
fprintf(stderr, "ERROR: Incorrect date/time or format. Format is %s.\n",
BackupAgentBase::timeFormat().c_str());
throw backup_error();
}
loop {
try {
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
state std::vector<std::pair<int64_t, Version>> results =
wait(versionMap.getRange(tr, 0, time, 1, false, true));
if (results.size() != 1) {
// No key less than time was found in the database
// Look for a key >= time.
wait(store(results, versionMap.getRange(tr, time, std::numeric_limits<int64_t>::max(), 1)));
if (results.size() != 1) {
fprintf(stderr, "ERROR: Unable to calculate a version for given date/time.\n");
throw backup_error();
}
}
// Adjust version found by the delta between time and the time found and min with 0.
auto& result = results[0];
return std::max<Version>(0, result.second + (time - result.first) * CLIENT_KNOBS->CORE_VERSIONSPERSECOND);
} catch (Error& e) {
wait(tr->onError(e));
}
}
}
ACTOR Future<Optional<int64_t>> timeKeeperEpochsFromVersion(Version v, Reference<ReadYourWritesTransaction> tr) {
state KeyBackedMap<int64_t, Version> versionMap(timeKeeperPrefixRange.begin);
// Binary search to find the closest date with a version <= v
state int64_t min = 0;
state int64_t max = (int64_t)now();
state int64_t mid;
state std::pair<int64_t, Version> found;
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
loop {
mid = (min + max + 1) / 2; // ceiling
// Find the highest time < mid
state std::vector<std::pair<int64_t, Version>> results =
wait(versionMap.getRange(tr, min, mid, 1, false, true));
if (results.size() != 1) {
if (mid == min) {
// There aren't any records having a version < v, so just look for any record having a time < now
// and base a result on it
wait(store(results, versionMap.getRange(tr, 0, (int64_t)now(), 1)));
if (results.size() != 1) {
// There aren't any timekeeper records to base a result on so return nothing
return Optional<int64_t>();
}
found = results[0];
break;
}
min = mid;
continue;
}
found = results[0];
if (v < found.second) {
max = found.first;
} else {
if (found.first == min) {
break;
}
<<<<<<< HEAD
=======
}
if(progress != nullptr) {
progress->step = "Final metadata update";
progress->total = 0;
}
// Update the expiredEndVersion metadata to indicate that everything prior to that version has been
// successfully deleted if the current version is lower or missing
Optional<Version> metaExpiredEnd = wait(bc->expiredEndVersion().get());
if(metaExpiredEnd.orDefault(0) < expireEndVersion) {
wait(bc->expiredEndVersion().set(expireEndVersion));
}
return Void();
}
// Delete all data up to (but not including endVersion)
Future<Void> expireData(Version expireEndVersion, bool force, ExpireProgress* progress,
Version restorableBeginVersion) final {
return expireData_impl(Reference<BackupContainerFileSystem>::addRef(this), expireEndVersion, force, progress, restorableBeginVersion);
}
// 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 partition 0 is continuous and create a map of ranges to tags
std::map<std::pair<Version, Version>, int> tags; // range [begin, end] -> tags
if (!isContinuous(files, tagIndices[0], begin, end, &tags)) {
TraceEvent(SevWarn, "BackupFileNotContinuous")
.detail("Partition", 0)
.detail("RangeBegin", begin)
.detail("RangeEnd", end);
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, std::min(beginEnd.second - 1, end), nullptr)) {
TraceEvent(SevWarn, "BackupFileNotContinuous")
.detail("Partition", i)
.detail("RangeBegin", beginEnd.first)
.detail("RangeEnd", beginEnd.second);
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])) {
ASSERT(logs[j].fileSize <= logs[i].fileSize);
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 contiguousLogEnd for "desc" if larger
// than the "scanBegin" version.
static void updatePartitionedLogsContinuousEnd(BackupDescription* desc, const std::vector<LogFile>& logs,
const Version scanBegin, const Version scanEnd) {
if (logs.empty()) return;
Version snapshotBeginVersion = desc->snapshots.size() > 0 ? desc->snapshots[0].beginVersion : invalidVersion;
Version begin = std::max(scanBegin, desc->minLogBegin.get());
TraceEvent("ContinuousLogEnd")
.detail("ScanBegin", scanBegin)
.detail("ScanEnd", scanEnd)
.detail("Begin", begin)
.detail("ContiguousLogEnd", desc->contiguousLogEnd.get());
for (const auto& file : logs) {
if (file.beginVersion > begin) {
if (scanBegin > 0) return;
// scanBegin is 0
desc->minLogBegin = file.beginVersion;
begin = file.beginVersion;
}
Version ver = getPartitionedLogsContinuousEndVersion(logs, begin);
if (ver >= desc->contiguousLogEnd.get()) {
// contiguousLogEnd is not inclusive, so +1 here.
desc->contiguousLogEnd.get() = ver + 1;
TraceEvent("UpdateContinuousLogEnd").detail("Version", ver + 1);
if (ver > snapshotBeginVersion) 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);
ASSERT(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])) {
ASSERT(logs[indices.back()].fileSize <= logs[i].fileSize);
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);
}
TraceEvent("ContinuousLogEnd").detail("Begin", begin).detail("InitVersion", end);
// check partition 0 is continuous in [begin, end] and create a map of ranges to partitions
std::map<std::pair<Version, Version>, int> tags; // range [start, end] -> partitions
isContinuous(logs, tagIndices[0], begin, end, &tags);
if (tags.empty() || end <= begin) return 0;
end = std::min(end, tags.rbegin()->first.second);
TraceEvent("ContinuousLogEnd").detail("Partition", 0).detail("EndVersion", end).detail("Begin", begin);
// for each range in tags, check all partitions from 1 are continouous
Version lastEnd = begin;
for (const auto& [beginEnd, count] : tags) {
Version tagEnd = beginEnd.second; // This range's minimum continous partition 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);
TraceEvent("ContinuousLogEnd")
.detail("Partition", i)
.detail("EndVersion", tagEnd)
.detail("RangeBegin", beginEnd.first)
.detail("RangeEnd", beginEnd.second);
if (tagEnd == 0) return lastEnd == begin ? 0 : lastEnd;
}
if (tagEnd < beginEnd.second) {
return tagEnd;
}
lastEnd = beginEnd.second;
}
return end;
}
ACTOR static Future<KeyRange> getSnapshotFileKeyRange_impl(Reference<BackupContainerFileSystem> bc,
RangeFile file) {
state int readFileRetries = 0;
state bool beginKeySet = false;
state Key beginKey;
state Key endKey;
loop {
try {
state Reference<IAsyncFile> inFile = wait(bc->readFile(file.fileName));
beginKeySet = false;
state int64_t j = 0;
for (; j < file.fileSize; j += file.blockSize) {
int64_t len = std::min<int64_t>(file.blockSize, file.fileSize - j);
Standalone<VectorRef<KeyValueRef>> blockData =
wait(fileBackup::decodeRangeFileBlock(inFile, j, len));
if (!beginKeySet) {
beginKey = blockData.front().key;
beginKeySet = true;
}
endKey = blockData.back().key;
}
break;
} catch (Error& e) {
if (e.code() == error_code_restore_bad_read ||
e.code() == error_code_restore_unsupported_file_version ||
e.code() == error_code_restore_corrupted_data_padding) { // no retriable error
TraceEvent(SevError, "BackupContainerGetSnapshotFileKeyRange").error(e);
throw;
} else if (e.code() == error_code_http_request_failed || e.code() == error_code_connection_failed ||
e.code() == error_code_timed_out || e.code() == error_code_lookup_failed) {
// blob http request failure, retry
TraceEvent(SevWarnAlways, "BackupContainerGetSnapshotFileKeyRangeConnectionFailure")
.detail("Retries", ++readFileRetries)
.error(e);
wait(delayJittered(0.1));
} else {
TraceEvent(SevError, "BackupContainerGetSnapshotFileKeyRangeUnexpectedError").error(e);
throw;
}
}
}
return KeyRange(KeyRangeRef(beginKey, endKey));
}
Future<KeyRange> getSnapshotFileKeyRange(const RangeFile& file) final {
ASSERT(g_network->isSimulated());
return getSnapshotFileKeyRange_impl(Reference<BackupContainerFileSystem>::addRef(this), file);
}
ACTOR static Future<Optional<RestorableFileSet>> getRestoreSet_impl(Reference<BackupContainerFileSystem> bc,
Version targetVersion,
VectorRef<KeyRangeRef> keyRangesFilter) {
// Find the most recent keyrange snapshot through which we can restore filtered key ranges into targetVersion.
state std::vector<KeyspaceSnapshotFile> snapshots = wait(bc->listKeyspaceSnapshots());
state int i = snapshots.size() - 1;
for (; i >= 0; i--) {
// The smallest version of filtered range files >= snapshot beginVersion > targetVersion
if (targetVersion >= 0 && snapshots[i].beginVersion > targetVersion) {
continue;
}
state RestorableFileSet restorable;
state Version minKeyRangeVersion = MAX_VERSION;
state Version maxKeyRangeVersion = -1;
std::pair<std::vector<RangeFile>, std::map<std::string, KeyRange>> results =
wait(bc->readKeyspaceSnapshot(snapshots[i]));
// Old backup does not have metadata about key ranges and can not be filtered with key ranges.
if (keyRangesFilter.size() && results.second.empty() && !results.first.empty()) {
throw backup_not_filterable_with_key_ranges();
}
// Filter by keyRangesFilter.
if (keyRangesFilter.empty()) {
restorable.ranges = std::move(results.first);
restorable.keyRanges = std::move(results.second);
minKeyRangeVersion = snapshots[i].beginVersion;
maxKeyRangeVersion = snapshots[i].endVersion;
} else {
for (const auto& rangeFile : results.first) {
const auto& keyRange = results.second.at(rangeFile.fileName);
if (keyRange.intersects(keyRangesFilter)) {
restorable.ranges.push_back(rangeFile);
restorable.keyRanges[rangeFile.fileName] = keyRange;
minKeyRangeVersion = std::min(minKeyRangeVersion, rangeFile.version);
maxKeyRangeVersion = std::max(maxKeyRangeVersion, rangeFile.version);
}
}
// No range file matches 'keyRangesFilter'.
if (restorable.ranges.empty()) {
throw backup_not_overlapped_with_keys_filter();
}
}
// 'latestVersion' represents using the minimum restorable version in a snapshot.
restorable.targetVersion = targetVersion == latestVersion ? maxKeyRangeVersion : targetVersion;
// Any version < maxKeyRangeVersion is not restorable.
if (restorable.targetVersion < maxKeyRangeVersion) continue;
restorable.snapshot = snapshots[i];
// TODO: Reenable the sanity check after TooManyFiles error is resolved
if (false && g_network->isSimulated()) {
// Sanity check key ranges
state std::map<std::string, KeyRange>::iterator rit;
for (rit = restorable.keyRanges.begin(); rit != restorable.keyRanges.end(); rit++) {
auto it = std::find_if(restorable.ranges.begin(), restorable.ranges.end(),
[file = rit->first](const RangeFile f) { return f.fileName == file; });
ASSERT(it != restorable.ranges.end());
KeyRange result = wait(bc->getSnapshotFileKeyRange(*it));
ASSERT(rit->second.begin <= result.begin && rit->second.end >= result.end);
}
}
// No logs needed if there is a complete filtered key space snapshot at the target version.
if (minKeyRangeVersion == maxKeyRangeVersion && maxKeyRangeVersion == restorable.targetVersion) {
restorable.continuousBeginVersion = restorable.continuousEndVersion = invalidVersion;
TraceEvent("BackupContainerGetRestorableFilesWithoutLogs")
.detail("KeyRangeVersion", restorable.targetVersion)
.detail("NumberOfRangeFiles", restorable.ranges.size())
.detail("KeyRangesFilter", printable(keyRangesFilter));
return Optional<RestorableFileSet>(restorable);
}
// FIXME: check if there are tagged logs. for each tag, there is no version gap.
state std::vector<LogFile> logs;
state std::vector<LogFile> plogs;
wait(store(logs, bc->listLogFiles(minKeyRangeVersion, restorable.targetVersion, false)) &&
store(plogs, bc->listLogFiles(minKeyRangeVersion, restorable.targetVersion, true)));
if (plogs.size() > 0) {
logs.swap(plogs);
// 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, minKeyRangeVersion, restorable.targetVersion)) {
restorable.continuousBeginVersion = minKeyRangeVersion;
restorable.continuousEndVersion = restorable.targetVersion + 1; // not inclusive
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 <= minKeyRangeVersion) {
Version end = logs.begin()->endVersion;
computeRestoreEndVersion(logs, &restorable.logs, &end, restorable.targetVersion);
if (end >= restorable.targetVersion) {
restorable.continuousBeginVersion = logs.begin()->beginVersion;
restorable.continuousEndVersion = end;
return Optional<RestorableFileSet>(restorable);
}
}
}
return Optional<RestorableFileSet>();
}
Future<Optional<RestorableFileSet>> getRestoreSet(Version targetVersion,
VectorRef<KeyRangeRef> keyRangesFilter) final {
return getRestoreSet_impl(Reference<BackupContainerFileSystem>::addRef(this), targetVersion, keyRangesFilter);
}
private:
struct VersionProperty {
VersionProperty(Reference<BackupContainerFileSystem> bc, std::string name) : bc(bc), path("properties/" + name) {}
Reference<BackupContainerFileSystem> bc;
std::string path;
Future<Optional<Version>> get() {
return readVersionProperty(bc, path);
}
Future<Void> set(Version v) {
return writeVersionProperty(bc, path, v);
}
Future<Void> clear() {
return bc->deleteFile(path);
}
};
public:
// To avoid the need to scan the underyling filesystem in many cases, some important version boundaries are stored in named files.
// These versions also indicate what version ranges are known to be deleted or partially deleted.
//
// The values below describe version ranges as follows:
// 0 - expiredEndVersion All files in this range have been deleted
// expiredEndVersion - unreliableEndVersion Some files in this range may have been deleted.
//
// logBeginVersion - logEnd Log files are contiguous in this range and have NOT been deleted by fdbbackup
// logEnd - infinity Files in this range may or may not exist yet
//
VersionProperty logBeginVersion() { return {Reference<BackupContainerFileSystem>::addRef(this), "log_begin_version"}; }
VersionProperty logEndVersion() { return {Reference<BackupContainerFileSystem>::addRef(this), "log_end_version"}; }
VersionProperty expiredEndVersion() { return {Reference<BackupContainerFileSystem>::addRef(this), "expired_end_version"}; }
VersionProperty unreliableEndVersion() { return {Reference<BackupContainerFileSystem>::addRef(this), "unreliable_end_version"}; }
// Backup log types
const static Version NON_PARTITIONED_MUTATION_LOG = 0;
const static Version PARTITIONED_MUTATION_LOG = 1;
VersionProperty logType() { return { Reference<BackupContainerFileSystem>::addRef(this), "mutation_log_type" }; }
ACTOR static Future<Void> writeVersionProperty(Reference<BackupContainerFileSystem> bc, std::string path, Version v) {
try {
state Reference<IBackupFile> f = wait(bc->writeFile(path));
std::string s = format("%lld", v);
wait(f->append(s.data(), s.size()));
wait(f->finish());
return Void();
} catch(Error &e) {
TraceEvent(SevWarn, "BackupContainerWritePropertyFailed")
.error(e)
.detail("URL", bc->getURL())
.detail("Path", path);
throw;
}
}
ACTOR static Future<Optional<Version>> readVersionProperty(Reference<BackupContainerFileSystem> bc, std::string path) {
try {
state Reference<IAsyncFile> f = wait(bc->readFile(path));
state int64_t size = wait(f->size());
state std::string s;
s.resize(size);
int rs = wait(f->read((uint8_t *)s.data(), size, 0));
Version v;
int len;
if(rs == size && sscanf(s.c_str(), "%" SCNd64 "%n", &v, &len) == 1 && len == size)
return v;
TraceEvent(SevWarn, "BackupContainerInvalidProperty")
.detail("URL", bc->getURL())
.detail("Path", path);
throw backup_invalid_info();
} catch(Error &e) {
if(e.code() == error_code_file_not_found)
return Optional<Version>();
TraceEvent(SevWarn, "BackupContainerReadPropertyFailed")
.error(e)
.detail("URL", bc->getURL())
.detail("Path", path);
throw;
}
}
};
class BackupContainerLocalDirectory : public BackupContainerFileSystem, ReferenceCounted<BackupContainerLocalDirectory> {
public:
void addref() final { return ReferenceCounted<BackupContainerLocalDirectory>::addref(); }
void delref() final { return ReferenceCounted<BackupContainerLocalDirectory>::delref(); }
static std::string getURLFormat() { return "file://</path/to/base/dir/>"; }
BackupContainerLocalDirectory(std::string url) {
std::string path;
if(url.find("file://") != 0) {
TraceEvent(SevWarn, "BackupContainerLocalDirectory").detail("Description", "Invalid URL for BackupContainerLocalDirectory").detail("URL", url);
}
path = url.substr(7);
// Remove trailing slashes on path
path.erase(path.find_last_not_of("\\/") + 1);
if(!g_network->isSimulated() && path != abspath(path)) {
TraceEvent(SevWarn, "BackupContainerLocalDirectory").detail("Description", "Backup path must be absolute (e.g. file:///some/path)").detail("URL", url).detail("Path", path);
throw io_error();
}
// Finalized path written to will be will be <path>/backup-<uid>
m_path = path;
}
static Future<std::vector<std::string>> listURLs(std::string url) {
std::string path;
if(url.find("file://") != 0) {
TraceEvent(SevWarn, "BackupContainerLocalDirectory").detail("Description", "Invalid URL for BackupContainerLocalDirectory").detail("URL", url);
}
path = url.substr(7);
// Remove trailing slashes on path
path.erase(path.find_last_not_of("\\/") + 1);
if(!g_network->isSimulated() && path != abspath(path)) {
TraceEvent(SevWarn, "BackupContainerLocalDirectory").detail("Description", "Backup path must be absolute (e.g. file:///some/path)").detail("URL", url).detail("Path", path);
throw io_error();
}
std::vector<std::string> dirs = platform::listDirectories(path);
std::vector<std::string> results;
for(auto &r : dirs) {
if(r == "." || r == "..")
continue;
results.push_back(std::string("file://") + joinPath(path, r));
}
return results;
}
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,
// which is done in openFile().
// Creating the directory here will result in unnecessary directories being created on machines that run fdbbackup but not agents.
return Void();
}
// The container exists if the folder it resides in exists
Future<bool> exists() final {
return directoryExists(m_path);
}
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
// but only if the source directory is writeable which shouldn't be required for a restore.
std::string fullPath = joinPath(m_path, path);
#ifndef _WIN32
if(g_network->isSimulated()) {
if(!fileExists(fullPath)) {
throw file_not_found();
}
if (g_simulator.getCurrentProcess()->uid == UID()) {
TraceEvent(SevError, "BackupContainerReadFileOnUnsetProcessID");
}
std::string uniquePath = fullPath + "." + g_simulator.getCurrentProcess()->uid.toString() + ".lnk";
unlink(uniquePath.c_str());
ASSERT(symlink(basename(path).c_str(), uniquePath.c_str()) == 0);
fullPath = uniquePath;
}
// Opening cached mode forces read/write mode at a lower level, overriding the readonly request. So cached mode
// can't be used because backup files are read-only. Cached mode can only help during restore task retries handled
// by the same process that failed the first task execution anyway, which is a very rare case.
#endif
Future<Reference<IAsyncFile>> f = IAsyncFileSystem::filesystem()->open(fullPath, flags, 0644);
if(g_network->isSimulated()) {
int blockSize = 0;
// Extract block size from the filename, if present
size_t lastComma = path.find_last_of(',');
if (lastComma != path.npos) {
blockSize = atoi(path.substr(lastComma + 1).c_str());
}
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);
int reads = deterministicRandom()->randomInt(1, 3);
int cacheSize = deterministicRandom()->randomInt(0, 3);
return Reference<IAsyncFile>(new AsyncFileReadAheadCache(fr, blockSize, readAhead, reads, cacheSize));
});
}
return f;
}
class BackupFile : public IBackupFile, ReferenceCounted<BackupFile> {
public:
BackupFile(std::string fileName, Reference<IAsyncFile> file, std::string finalFullPath)
: IBackupFile(fileName), m_file(file), m_finalFullPath(finalFullPath), m_writeOffset(0)
{
m_buffer.reserve(m_buffer.arena(), CLIENT_KNOBS->BACKUP_LOCAL_FILE_WRITE_BLOCK);
}
Future<Void> append(const void *data, int len) {
m_buffer.append(m_buffer.arena(), (const uint8_t *)data, len);
if(m_buffer.size() >= CLIENT_KNOBS->BACKUP_LOCAL_FILE_WRITE_BLOCK) {
return flush(CLIENT_KNOBS->BACKUP_LOCAL_FILE_WRITE_BLOCK);
}
return Void();
}
Future<Void> flush(int size) {
ASSERT(size <= m_buffer.size());
// Keep a reference to the old buffer
Standalone<VectorRef<uint8_t>> old = m_buffer;
// Make a new buffer, initialized with the excess bytes over the block size from the old buffer
m_buffer = Standalone<VectorRef<uint8_t>>(old.slice(size, old.size()));
// Write the old buffer to the underlying file and update the write offset
Future<Void> r = holdWhile(old, m_file->write(old.begin(), size, m_writeOffset));
m_writeOffset += size;
return r;
}
ACTOR static Future<Void> finish_impl(Reference<BackupFile> f) {
wait(f->flush(f->m_buffer.size()));
wait(f->m_file->truncate(f->size())); // Some IAsyncFile implementations extend in whole block sizes.
wait(f->m_file->sync());
std::string name = f->m_file->getFilename();
f->m_file.clear();
renameFile(name, f->m_finalFullPath);
return Void();
}
int64_t size() const {
return m_buffer.size() + m_writeOffset;
}
Future<Void> finish() {
return finish_impl(Reference<BackupFile>::addRef(this));
}
void addref() override { return ReferenceCounted<BackupFile>::addref(); }
void delref() override { return ReferenceCounted<BackupFile>::delref(); }
private:
Reference<IAsyncFile> m_file;
Standalone<VectorRef<uint8_t>> m_buffer;
int64_t m_writeOffset;
std::string m_finalFullPath;
};
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));
std::string temp = fullPath + "." + deterministicRandom()->randomUniqueID().toString() + ".temp";
Future<Reference<IAsyncFile>> f = IAsyncFileSystem::filesystem()->open(temp, flags, 0644);
return map(f, [=](Reference<IAsyncFile> f) {
return Reference<IBackupFile>(new BackupFile(path, f, fullPath));
});
}
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&)>) final {
FilesAndSizesT results;
std::vector<std::string> files;
platform::findFilesRecursively(joinPath(m_path, path), files);
// Remove .lnk files from results, they are a side effect of a backup that was *read* during simulation. See openFile() above for more info on why they are created.
if(g_network->isSimulated())
files.erase(std::remove_if(files.begin(), files.end(), [](std::string const &f) { return StringRef(f).endsWith(LiteralStringRef(".lnk")); }), files.end());
for(auto &f : files) {
// Hide .part or .temp files.
StringRef s(f);
if(!s.endsWith(LiteralStringRef(".part")) && !s.endsWith(LiteralStringRef(".temp")))
results.push_back({f.substr(m_path.size() + 1), ::fileSize(f)});
}
return results;
}
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) {
// If the backup has no snapshots and no logs then it's probably not a valid backup
if(desc.snapshots.size() == 0 && !desc.minLogBegin.present())
throw backup_invalid_url();
int count = platform::eraseDirectoryRecursive(m_path);
if(pNumDeleted != nullptr)
*pNumDeleted = count;
return Void();
});
}
private:
std::string m_path;
};
class BackupContainerBlobStore : public BackupContainerFileSystem, ReferenceCounted<BackupContainerBlobStore> {
private:
// Backup files to under a single folder prefix with subfolders for each named backup
static const std::string DATAFOLDER;
// Indexfolder contains keys for which user-named backups exist. Backup names can contain an arbitrary
// number of slashes so the backup names are kept in a separate folder tree from their actual data.
static const std::string INDEXFOLDER;
Reference<BlobStoreEndpoint> m_bstore;
std::string m_name;
// All backup data goes into a single bucket
std::string m_bucket;
std::string dataPath(const std::string path) {
return DATAFOLDER + "/" + m_name + "/" + path;
}
// Get the path of the backups's index entry
std::string indexEntry() {
return INDEXFOLDER + "/" + m_name;
}
public:
BackupContainerBlobStore(Reference<BlobStoreEndpoint> bstore, std::string name, const BlobStoreEndpoint::ParametersT &params)
: m_bstore(bstore), m_name(name), m_bucket("FDB_BACKUPS_V2") {
// Currently only one parameter is supported, "bucket"
for(auto &kv : params) {
if(kv.first == "bucket") {
m_bucket = kv.second;
continue;
}
TraceEvent(SevWarn, "BackupContainerBlobStoreInvalidParameter").detail("Name", kv.first).detail("Value", kv.second);
IBackupContainer::lastOpenError = format("Unknown URL parameter: '%s'", kv.first.c_str());
throw backup_invalid_url();
}
}
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.)";
}
virtual ~BackupContainerBlobStore() {}
Future<Reference<IAsyncFile>> readFile(std::string path) final {
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) {
state std::string basePath = INDEXFOLDER + '/';
BlobStoreEndpoint::ListResult contents = wait(bstore->listObjects(bucket, basePath));
std::vector<std::string> results;
for(auto &f : contents.objects) {
results.push_back(bstore->getResourceURL(f.name.substr(basePath.size()), format("bucket=%s", bucket.c_str())));
}
return results;
}
class BackupFile : public IBackupFile, ReferenceCounted<BackupFile> {
public:
BackupFile(std::string fileName, Reference<IAsyncFile> file) : IBackupFile(fileName), m_file(file), m_offset(0) {}
Future<Void> append(const void *data, int len) {
Future<Void> r = m_file->write(data, len, m_offset);
m_offset += len;
return r;
}
Future<Void> finish() {
Reference<BackupFile> self = Reference<BackupFile>::addRef(this);
return map(m_file->sync(), [=](Void _) { self->m_file.clear(); return Void(); });
}
int64_t size() const {
return m_offset;
}
void addref() final { return ReferenceCounted<BackupFile>::addref(); }
void delref() final { return ReferenceCounted<BackupFile>::delref(); }
private:
Reference<IAsyncFile> m_file;
int64_t m_offset;
};
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) final {
return m_bstore->deleteObject(m_bucket, dataPath(path));
}
ACTOR static Future<FilesAndSizesT> listFiles_impl(Reference<BackupContainerBlobStore> bc, std::string path, std::function<bool(std::string const &)> pathFilter) {
// pathFilter expects container based paths, so create a wrapper which converts a raw path
// to a container path by removing the known backup name prefix.
state int prefixTrim = bc->dataPath("").size();
std::function<bool(std::string const &)> rawPathFilter = [=](const std::string &folderPath) {
ASSERT(folderPath.size() >= prefixTrim);
return pathFilter(folderPath.substr(prefixTrim));
};
state BlobStoreEndpoint::ListResult result = wait(bc->m_bstore->listObjects(bc->m_bucket, bc->dataPath(path), '/', std::numeric_limits<int>::max(), rawPathFilter));
FilesAndSizesT files;
for(auto &o : result.objects) {
ASSERT(o.name.size() >= prefixTrim);
files.push_back({o.name.substr(prefixTrim), o.size});
}
return files;
}
Future<FilesAndSizesT> listFiles(std::string path, std::function<bool(std::string const &)> pathFilter) final {
return listFiles_impl(Reference<BackupContainerBlobStore>::addRef(this), path, pathFilter);
}
ACTOR static Future<Void> create_impl(Reference<BackupContainerBlobStore> bc) {
wait(bc->m_bstore->createBucket(bc->m_bucket));
// Check/create the index entry
bool exists = wait(bc->m_bstore->objectExists(bc->m_bucket, bc->indexEntry()));
if(!exists) {
wait(bc->m_bstore->writeEntireFile(bc->m_bucket, bc->indexEntry(), ""));
}
return Void();
}
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() final {
return m_bstore->objectExists(m_bucket, indexEntry());
}
ACTOR static Future<Void> deleteContainer_impl(Reference<BackupContainerBlobStore> bc, int *pNumDeleted) {
bool e = wait(bc->exists());
if(!e) {
TraceEvent(SevWarnAlways, "BackupContainerDoesNotExist").detail("URL", bc->getURL());
throw backup_does_not_exist();
}
// First delete everything under the data prefix in the bucket
wait(bc->m_bstore->deleteRecursively(bc->m_bucket, bc->dataPath(""), pNumDeleted));
// Now that all files are deleted, delete the index entry
wait(bc->m_bstore->deleteObject(bc->m_bucket, bc->indexEntry()));
return Void();
}
Future<Void> deleteContainer(int* pNumDeleted) final {
return deleteContainer_impl(Reference<BackupContainerBlobStore>::addRef(this), pNumDeleted);
}
std::string getBucket() const {
return m_bucket;
}
};
const std::string BackupContainerBlobStore::DATAFOLDER = "data";
const std::string BackupContainerBlobStore::INDEXFOLDER = "backups";
std::string IBackupContainer::lastOpenError;
std::vector<std::string> IBackupContainer::getURLFormats() {
std::vector<std::string> formats;
formats.push_back(BackupContainerLocalDirectory::getURLFormat());
formats.push_back(BackupContainerBlobStore::getURLFormat());
return formats;
}
// Get an IBackupContainer based on a container URL string
Reference<IBackupContainer> IBackupContainer::openContainer(std::string url)
{
static std::map<std::string, Reference<IBackupContainer>> m_cache;
Reference<IBackupContainer> &r = m_cache[url];
if(r)
return r;
try {
StringRef u(url);
if(u.startsWith(LiteralStringRef("file://")))
r = Reference<IBackupContainer>(new BackupContainerLocalDirectory(url));
else if(u.startsWith(LiteralStringRef("blobstore://"))) {
std::string resource;
// The URL parameters contain blobstore endpoint tunables as well as possible backup-specific options.
BlobStoreEndpoint::ParametersT backupParams;
Reference<BlobStoreEndpoint> bstore = BlobStoreEndpoint::fromString(url, &resource, &lastOpenError, &backupParams);
if(resource.empty())
throw backup_invalid_url();
for(auto c : resource)
if(!isalnum(c) && c != '_' && c != '-' && c != '.' && c != '/')
throw backup_invalid_url();
r = Reference<IBackupContainer>(new BackupContainerBlobStore(bstore, resource, backupParams));
}
else {
lastOpenError = "invalid URL prefix";
throw backup_invalid_url();
}
r->URL = url;
return r;
} catch(Error &e) {
if(e.code() == error_code_actor_cancelled)
throw;
TraceEvent m(SevWarn, "BackupContainer");
m.detail("Description", "Invalid container specification. See help.");
m.detail("URL", url);
m.error(e);
if(e.code() == error_code_backup_invalid_url)
m.detail("LastOpenError", lastOpenError);
throw;
}
}
// Get a list of URLS to backup containers based on some a shorter URL. This function knows about some set of supported
// URL types which support this sort of backup discovery.
ACTOR Future<std::vector<std::string>> listContainers_impl(std::string baseURL) {
try {
StringRef u(baseURL);
if(u.startsWith(LiteralStringRef("file://"))) {
std::vector<std::string> results = wait(BackupContainerLocalDirectory::listURLs(baseURL));
return results;
}
else if(u.startsWith(LiteralStringRef("blobstore://"))) {
std::string resource;
BlobStoreEndpoint::ParametersT backupParams;
Reference<BlobStoreEndpoint> bstore = BlobStoreEndpoint::fromString(baseURL, &resource, &IBackupContainer::lastOpenError, &backupParams);
if(!resource.empty()) {
TraceEvent(SevWarn, "BackupContainer").detail("Description", "Invalid backup container base URL, resource aka path should be blank.").detail("URL", baseURL);
throw backup_invalid_url();
}
// Create a dummy container to parse the backup-specific parameters from the URL and get a final bucket name
BackupContainerBlobStore dummy(bstore, "dummy", backupParams);
std::vector<std::string> results = wait(BackupContainerBlobStore::listURLs(bstore, dummy.getBucket()));
return results;
}
else {
IBackupContainer::lastOpenError = "invalid URL prefix";
throw backup_invalid_url();
}
} catch(Error &e) {
if(e.code() == error_code_actor_cancelled)
throw;
TraceEvent m(SevWarn, "BackupContainer");
m.detail("Description", "Invalid backup container URL prefix. See help.");
m.detail("URL", baseURL);
m.error(e);
if(e.code() == error_code_backup_invalid_url)
m.detail("LastOpenError", IBackupContainer::lastOpenError);
throw;
}
}
Future<std::vector<std::string>> IBackupContainer::listContainers(std::string baseURL) {
return listContainers_impl(baseURL);
}
ACTOR Future<Version> timeKeeperVersionFromDatetime(std::string datetime, Database db) {
state KeyBackedMap<int64_t, Version> versionMap(timeKeeperPrefixRange.begin);
state Reference<ReadYourWritesTransaction> tr = Reference<ReadYourWritesTransaction>(new ReadYourWritesTransaction(db));
state int64_t time = BackupAgentBase::parseTime(datetime);
if(time < 0) {
fprintf(stderr, "ERROR: Incorrect date/time or format. Format is %s.\n", BackupAgentBase::timeFormat().c_str());
throw backup_error();
}
loop {
try {
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
state std::vector<std::pair<int64_t, Version>> results = wait( versionMap.getRange(tr, 0, time, 1, false, true) );
if (results.size() != 1) {
// No key less than time was found in the database
// Look for a key >= time.
wait( store( results, versionMap.getRange(tr, time, std::numeric_limits<int64_t>::max(), 1) ) );
if(results.size() != 1) {
fprintf(stderr, "ERROR: Unable to calculate a version for given date/time.\n");
throw backup_error();
}
}
// Adjust version found by the delta between time and the time found and min with 0.
auto &result = results[0];
return std::max<Version>(0, result.second + (time - result.first) * CLIENT_KNOBS->CORE_VERSIONSPERSECOND);
} catch (Error& e) {
wait(tr->onError(e));
}
}
}
ACTOR Future<Optional<int64_t>> timeKeeperEpochsFromVersion(Version v, Reference<ReadYourWritesTransaction> tr) {
state KeyBackedMap<int64_t, Version> versionMap(timeKeeperPrefixRange.begin);
// Binary search to find the closest date with a version <= v
state int64_t min = 0;
state int64_t max = (int64_t)now();
state int64_t mid;
state std::pair<int64_t, Version> found;
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
loop {
mid = (min + max + 1) / 2; // ceiling
// Find the highest time < mid
state std::vector<std::pair<int64_t, Version>> results = wait( versionMap.getRange(tr, min, mid, 1, false, true) );
if (results.size() != 1) {
if(mid == min) {
// There aren't any records having a version < v, so just look for any record having a time < now
// and base a result on it
wait(store(results, versionMap.getRange(tr, 0, (int64_t)now(), 1)));
if (results.size() != 1) {
// There aren't any timekeeper records to base a result on so return nothing
return Optional<int64_t>();
}
found = results[0];
break;
}
min = mid;
continue;
}
found = results[0];
if(v < found.second) {
max = found.first;
}
else {
if(found.first == min) {
break;
}
>>>>>>> release-6.3
min = found.first;
}
}
return found.first + (v - found.second) / CLIENT_KNOBS->CORE_VERSIONSPERSECOND;
}